You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2016/08/12 16:06:18 UTC

[01/50] [abbrv] incubator-quickstep git commit: Cleaned up the messages w/ a dummy payload. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/travis-grpc a6464d8a7 -> 9c9f32869 (forced update)


Cleaned up the messages w/ a dummy payload.


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

Branch: refs/heads/travis-grpc
Commit: ccea2ff83ea73e950d52c152cc422a9e93cf6aad
Parents: 52a32a3
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Fri Jul 29 23:52:33 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 1 10:11:19 2016 -0700

----------------------------------------------------------------------
 query_execution/ForemanSingleNode.cpp           |   4 +-
 query_execution/QueryExecutionMessages.proto    |   4 -
 query_execution/QueryExecutionUtil.hpp          |   5 +-
 query_execution/README.md                       | 110 +++++++++----------
 query_execution/WorkerMessage.hpp               |  13 +--
 query_execution/tests/BlockLocator_unittest.cpp |  11 +-
 storage/tests/DataExchange_unittest.cpp         |  11 +-
 7 files changed, 60 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index cda02a7..d2b56ae 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -167,9 +167,7 @@ void ForemanSingleNode::run() {
     if (!policy_enforcer_->hasQueries()) {
       // Signal the main thread that there are no queries to be executed.
       // Currently the message doesn't have any real content.
-      const int dummy_payload = 0;
-      TaggedMessage completion_tagged_message(
-          &dummy_payload, sizeof(dummy_payload), kWorkloadCompletionMessage);
+      TaggedMessage completion_tagged_message(kWorkloadCompletionMessage);
       const tmb::MessageBus::SendStatus send_status =
           QueryExecutionUtil::SendTMBMessage(
               bus_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 308d736..f2219f6 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -20,10 +20,6 @@ import "catalog/Catalog.proto";
 import "query_execution/QueryContext.proto";
 import "relational_operators/WorkOrder.proto";
 
-// Used for any messages that do not carry payloads.
-message EmptyMessage {
-}
-
 // Note: There are different types of completion messages for normal work orders
 // rebuild work orders. This can be potentially helpful when we want to collect
 // different statistics for executing different types of work orders.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index 6ea4a29..5994f22 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -123,10 +123,7 @@ class QueryExecutionUtil {
     style.Broadcast(true);
     Address address;
     address.All(true);
-    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-    TaggedMessage poison_tagged_message(poison_message.get(),
-                                        sizeof(*poison_message),
-                                        kPoisonMessage);
+    TaggedMessage poison_tagged_message(kPoisonMessage);
 
     const tmb::MessageBus::SendStatus send_status = bus->Send(
         sender_id, address, style, std::move(poison_tagged_message));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/query_execution/README.md
----------------------------------------------------------------------
diff --git a/query_execution/README.md b/query_execution/README.md
index 22ad91d..12e0f57 100644
--- a/query_execution/README.md
+++ b/query_execution/README.md
@@ -3,19 +3,19 @@
 ## Types of threads
 There are two kinds of threads in Quickstep - Foreman and Worker. The foreman
 thread controls the query execution progress, finds schedulable work (called as
-WorkOrder) and assigns (or schedules) it for execution to the Worker threads. 
+WorkOrder) and assigns (or schedules) it for execution to the Worker threads.
 The Worker threads receive the WorkOrders and execute them. After execution they
-send a completion message (or response message) back to Foreman. 
+send a completion message (or response message) back to Foreman.
 
 ## High level functionality of Foreman
-Foreman requests all the RelationalOperators in the physical query plan 
+Foreman requests all the RelationalOperators in the physical query plan
 represented as a DAG to give any schedulable work (in the form of WorkOrders).
 While doing so, Foreman has to respect dependencies between operators. There are
-two kinds of dependencies between operators - pipeline breaking (or blocking) 
+two kinds of dependencies between operators - pipeline breaking (or blocking)
 and pipeline non-breaking (or non-blocking). In the first case, the output of
 the producer operator can't be pipelined to the consumer operator. In the second
 case, the Foreman will facilitate the pipelining of the intermediate output
-produced by the producer operator to the consumer operator. 
+produced by the producer operator to the consumer operator.
 
 ## Messages in execution engine
 
@@ -26,110 +26,110 @@ of the message.
 Foreman -> Worker : WorkerMessage which consists of the following things
 - A pointer to the WorkOrder to be executed. The WorkOrder could be a normal
 WorkOrder or a rebuild WorkOrder. A normal WorkOrder involves the invocation of
-WorkOrder::execute() method which is overriden by all of the RelationalOperator 
-classes. A rebuild WorkOrder has one StorageBlock as input and calls a 
+WorkOrder::execute() method which is overriden by all of the RelationalOperator
+classes. A rebuild WorkOrder has one StorageBlock as input and calls a
 rebuild() method on the block. More details about rebuild() can be found in the
-storage module. 
+storage module.
 - The index of the relational operator in the query plan DAG that produced the
-WorkOrder. 
-
-Main thread -> Worker : WorkerMessage of type PoisonMessage. This message is
-used to terminate the Worker thread, typically when shutting down the Quickstep
-process. 
+WorkOrder.
 
 ### ForemanMessage
-Multiple senders are possible for this message. There are multiple types of 
+Multiple senders are possible for this message. There are multiple types of
 ForemanMessages, each of which indicates the purpose of the message.
 
-Worker -> Foreman : ForemanMessage of types WorkOrderCompletion and 
-RebuildCompletion are sent after a Worker finishes executing a respective type 
-of WorkOrder. This message helps the Foreman track the progress of individual 
-operators as well as the whole query. 
+Worker -> Foreman : ForemanMessage of types WorkOrderCompletion and
+RebuildCompletion are sent after a Worker finishes executing a respective type
+of WorkOrder. This message helps the Foreman track the progress of individual
+operators as well as the whole query.
 
 Some relational operators and InsertDestination -> Foreman : ForemanMessage of
-types DataPipeline and WorkOrdersAvailable. InsertDestination first determines 
+types DataPipeline and WorkOrdersAvailable. InsertDestination first determines
 when an output block of a relational operator gets full. Once a block is full,
-it streams the unique block ID of the filled block along with the index of the 
-relational operator that produced the block to Foreman with the message type 
+it streams the unique block ID of the filled block along with the index of the
+relational operator that produced the block to Foreman with the message type
 DataPipeline. Some operators which modify the block in place also send similar
-messages to Foreman. 
+messages to Foreman.
 
 ### FeedbackMessage
 This message is sent from Workers to the Foreman during a WorkOrder execution.
 
 In certain operators, e.g. TextScan (used for bulk loading data from text files)
-and Sort, there is a communication between the relational operator and its 
-WorkOrders. In such cases, when a WorkOrder is under execution on a Worker 
+and Sort, there is a communication between the relational operator and its
+WorkOrders. In such cases, when a WorkOrder is under execution on a Worker
 thread, a FeedbackMessage is sent from the WorkOrder via the Worker to Foreman.
 Foreman relays this message to the relational operator that produced the sender
-WorkOrder. The relational operator uses this message to update its internal 
-state to potentially generate newer WorkOrders. 
+WorkOrder. The relational operator uses this message to update its internal
+state to potentially generate newer WorkOrders.
+
+### PoisonMessage
+This message is used to terminate a thread (i.e., Foreman and Worker), typically
+when shutting down the Quickstep process.
 
 ## How does the Foreman react after receiving various messages?
 ### WorkOrder completion message
 * Update the book-keeping of pending WorkOrders per Worker and per operator.
 * Fetch new WorkOrders if available for the operator of whose WorkOrder was
-just executed.  
+just executed.
 * Update the state of an operator - the possible options are:
   - Normal WorkOrders are still under execution
   - All normal WorkOrders have finished execution and rebuild WorkOrders are yet
-  to be generated. 
+  to be generated.
   - All normal WorkOrders have finished execution, rebuild WorkOrders have been
-  generated and issued to Workers. 
+  generated and issued to Workers.
   - All normal and rebuild WorkOrders have been executed AND all the dependency
-  operators for the given operator have finished execution, therefore the given 
-  operator has finished its execution. 
-* Fetch the WorkOrders from the dependents of the given operator. 
+  operators for the given operator have finished execution, therefore the given
+  operator has finished its execution.
+* Fetch the WorkOrders from the dependents of the given operator.
 
 ### Rebuild WorkOrder completion message
 * Update the book-keeping of pending WorkOrders per Worker and per operator.
 * If all the rebuild WorkOrders have finished their execution, try to fetch the
 WorkOrders of the dependent operators of the operator whose rebuild WorkOrder
-was just executed. 
+was just executed.
 
 ### Data pipeline message
-* Find the consumer operators (i.e. operators which have a non 
-pipeline-breaking link) of the producer operator. 
-* Stream the block ID to the eligible consumer operators. 
-* Fetch new WorkOrders from these consumer operators which may have become 
-available because of the streaming of data. 
+* Find the consumer operators (i.e. operators which have a non
+pipeline-breaking link) of the producer operator.
+* Stream the block ID to the eligible consumer operators.
+* Fetch new WorkOrders from these consumer operators which may have become
+available because of the streaming of data.
 
 ### WorkOrder available message
 * Fetch new WorkOrders that may have become available.
 
 ### Feedback message
-* Relay the feedback message to a specified relational operator. The recipient 
-operator is specified in the header of the message. 
+* Relay the feedback message to a specified relational operator. The recipient
+operator is specified in the header of the message.
 
 ## Example
-We look at a sample query to better describe the flow of messages - 
+We look at a sample query to better describe the flow of messages -
 
 SELECT R.a, S.b from R, S where R.a = S.a and R.c < 20;
 
-This is an equi-join query which can be implemented using a hash join. We assume 
-that S is a larger relation and the build relation is the output of the 
+This is an equi-join query which can be implemented using a hash join. We assume
+that S is a larger relation and the build relation is the output of the
 selection on R.
 
 The query execution plan involves the following operators:
-* SelectOperator to filter R based on predicate R.c < 20 (We call the output as 
-R') 
+* SelectOperator to filter R based on predicate R.c < 20 (We call the output as
+R')
 * BuildHashOperator to construct a hash table on R'
 * HashJoinOperator to probe the hash table, where the probe relation is S
 * DestroyHashTableOperator to destroy the hash table after the join is done
-* Multiple DropTableOperators to destroy the temporaray relations produced as 
-output. 
+* Multiple DropTableOperators to destroy the temporaray relations produced as
+output.
 
 R has two blocks with IDs as 1 and 2. S has two blocks with IDs as 3 and 4.
-We assume that the SelectOperator produces one filled block and one partially 
-filled block as output. Note that in the query plan DAG, the link between 
-SelectOperator and BuildHashOperator allows streaming of data. The 
-HashJoinOperator's WorkOrder can't be generated unless all of the 
+We assume that the SelectOperator produces one filled block and one partially
+filled block as output. Note that in the query plan DAG, the link between
+SelectOperator and BuildHashOperator allows streaming of data. The
+HashJoinOperator's WorkOrder can't be generated unless all of the
 BuildHashOperator's WorkOrders have finished their execution. The execution is
-assumed to be performed by a single Worker thread. 
+assumed to be performed by a single Worker thread.
 
-The following table describes the message exchange that happens during the 
-query excution. We primarily focus on three operators - Select, BuildHash and 
-HashJoin (probe). 
+The following table describes the message exchange that happens during the
+query excution. We primarily focus on three operators - Select, BuildHash and
+HashJoin (probe).
 
 | Sender | Receiver | Message  | Message Description |
 |:-----------------:|----------|---------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/query_execution/WorkerMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkerMessage.hpp b/query_execution/WorkerMessage.hpp
index 560c1ba..a0434de 100644
--- a/query_execution/WorkerMessage.hpp
+++ b/query_execution/WorkerMessage.hpp
@@ -35,7 +35,6 @@ class WorkerMessage {
   enum class WorkerMessageType {
     kRebuildWorkOrder = 0,
     kWorkOrder,
-    kPoison
   };
 
   /**
@@ -70,15 +69,6 @@ class WorkerMessage {
   }
 
   /**
-   * @brief A static factory method for generating a poison message.
-   *
-   * @return The constructed PoisonMessage.
-   **/
-  static WorkerMessage* PoisonMessage() {
-    return new WorkerMessage(nullptr, 0, WorkerMessageType::kPoison);
-  }
-
-  /**
    * @brief Destructor.
    **/
   ~WorkerMessage() {
@@ -128,8 +118,7 @@ class WorkerMessage {
   /**
    * @brief Constructor.
    *
-   * @param work_unit The work order to be executed by the worker. A NULL
-   *        workorder indicates a poison message.
+   * @param work_unit The work order to be executed by the worker.
    * @param relational_op_index The index of the relational operator in the
    *        query plan DAG that generated the given WorkOrder.
    * @param type Type of the WorkerMessage.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/query_execution/tests/BlockLocator_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/BlockLocator_unittest.cpp b/query_execution/tests/BlockLocator_unittest.cpp
index fe7b86b..fd25e9e 100644
--- a/query_execution/tests/BlockLocator_unittest.cpp
+++ b/query_execution/tests/BlockLocator_unittest.cpp
@@ -90,16 +90,7 @@ class BlockLocatorTest : public ::testing::Test {
   virtual void TearDown() {
     storage_manager_.reset();
 
-    serialization::EmptyMessage proto;
-
-    const int proto_length = proto.ByteSize();
-    char *proto_bytes = static_cast<char*>(malloc(proto_length));
-    CHECK(proto.SerializeToArray(proto_bytes, proto_length));
-
-    TaggedMessage message(static_cast<const void*>(proto_bytes),
-                          proto_length,
-                          kPoisonMessage);
-    free(proto_bytes);
+    TaggedMessage message(kPoisonMessage);
 
     LOG(INFO) << "Worker (id '" << worker_client_id_
               << "') sent PoisonMessage (typed '" << kPoisonMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ccea2ff8/storage/tests/DataExchange_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/DataExchange_unittest.cpp b/storage/tests/DataExchange_unittest.cpp
index 38d12f6..4bad17b 100644
--- a/storage/tests/DataExchange_unittest.cpp
+++ b/storage/tests/DataExchange_unittest.cpp
@@ -105,16 +105,7 @@ class DataExchangeTest : public ::testing::Test {
     data_exchanger_expected_.shutdown();
     storage_manager_expected_.reset();
 
-    serialization::EmptyMessage proto;
-
-    const int proto_length = proto.ByteSize();
-    char *proto_bytes = static_cast<char*>(malloc(proto_length));
-    CHECK(proto.SerializeToArray(proto_bytes, proto_length));
-
-    TaggedMessage message(static_cast<const void*>(proto_bytes),
-                          proto_length,
-                          kPoisonMessage);
-    free(proto_bytes);
+    TaggedMessage message(kPoisonMessage);
 
     LOG(INFO) << "Worker (id '" << worker_client_id_
               << "') sent PoisonMessage (typed '" << kPoisonMessage


[02/50] [abbrv] incubator-quickstep git commit: TMB: Added Support for an Empty, but Typed TaggedMessage.

Posted by zu...@apache.org.
TMB: Added Support for an Empty, but Typed TaggedMessage.


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

Branch: refs/heads/travis-grpc
Commit: 52a32a372612a3e8f5ab268886d4b9cbb546c205
Parents: 260b862
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Fri Jul 29 19:57:49 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 1 10:11:19 2016 -0700

----------------------------------------------------------------------
 third_party/tmb/include/tmb/tagged_message.h | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/52a32a37/third_party/tmb/include/tmb/tagged_message.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/tagged_message.h b/third_party/tmb/include/tmb/tagged_message.h
index 49dcee7..f3a77ee 100644
--- a/third_party/tmb/include/tmb/tagged_message.h
+++ b/third_party/tmb/include/tmb/tagged_message.h
@@ -63,6 +63,15 @@ class TaggedMessage {
   }
 
   /**
+   * @brief Constructor which creates an empty, but typed message.
+   **/
+  explicit TaggedMessage(const message_type_id message_type)
+      : payload_inline_(true),
+        message_type_(message_type) {
+    payload_.in_line.size = 0;
+  }
+
+  /**
    * @brief Constructor.
    *
    * @param msg A pointer to the message contents in memory, which will be


[31/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/PolicyEnforcerSingleNode.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerSingleNode.hpp b/query_execution/PolicyEnforcerSingleNode.hpp
index 671fd83..870df95 100644
--- a/query_execution/PolicyEnforcerSingleNode.hpp
+++ b/query_execution/PolicyEnforcerSingleNode.hpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_SINGLE_NODE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 7019b6a..2572e18 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/QueryContext.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 9171250..c54c7ff 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_CONTEXT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryContext.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.proto b/query_execution/QueryContext.proto
index ddf8326..1a586a4 100644
--- a/query_execution/QueryContext.proto
+++ b/query_execution/QueryContext.proto
@@ -1,17 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 4922042..f6a8b73 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -1,16 +1,19 @@
-//   Copyright 2015-2016 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 
@@ -38,8 +41,8 @@ message NormalWorkOrderCompletionMessage {
   required uint64 operator_index = 1;
   required uint64 worker_thread_index = 2;
   required uint64 query_id = 3;
-  
-  // Epoch time in microseconds.  
+
+  // Epoch time in microseconds.
   optional uint64 execution_start_time = 4;
   optional uint64 execution_end_time = 5;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryExecutionModule.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionModule.hpp b/query_execution/QueryExecutionModule.hpp
index b2ff30f..89979f1 100644
--- a/query_execution/QueryExecutionModule.hpp
+++ b/query_execution/QueryExecutionModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup QueryExecution

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryExecutionState.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionState.hpp b/query_execution/QueryExecutionState.hpp
index 7395461..9ae9563 100644
--- a/query_execution/QueryExecutionState.hpp
+++ b/query_execution/QueryExecutionState.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_STATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 4bbab59..4643096 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_TYPEDEFS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index 5994f22..906fb6b 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index d2a3341..c60e323 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/QueryManagerBase.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryManagerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.hpp b/query_execution/QueryManagerBase.hpp
index 6edfd5c..782b8ed 100644
--- a/query_execution/QueryManagerBase.hpp
+++ b/query_execution/QueryManagerBase.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_BASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 9e192eb..e300ce5 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 "query_execution/QueryManagerDistributed.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 3eee95c..b448528 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_DISTRIBUTED_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryManagerSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerSingleNode.cpp b/query_execution/QueryManagerSingleNode.cpp
index 12f8ff5..237796f 100644
--- a/query_execution/QueryManagerSingleNode.cpp
+++ b/query_execution/QueryManagerSingleNode.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/QueryManagerSingleNode.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/QueryManagerSingleNode.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerSingleNode.hpp b/query_execution/QueryManagerSingleNode.hpp
index b9bad2e..dd044a5 100644
--- a/query_execution/QueryManagerSingleNode.hpp
+++ b/query_execution/QueryManagerSingleNode.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_SINGLE_NODE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 50381b2..120e8fb 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 "query_execution/Shiftboss.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 9e24d62..32d2408 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/ShiftbossDirectory.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ShiftbossDirectory.hpp b/query_execution/ShiftbossDirectory.hpp
index abb5f32..88d3039 100644
--- a/query_execution/ShiftbossDirectory.hpp
+++ b/query_execution/ShiftbossDirectory.hpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_DIRECTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/WorkOrderProtosContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrderProtosContainer.hpp b/query_execution/WorkOrderProtosContainer.hpp
index 5043755..b037332 100644
--- a/query_execution/WorkOrderProtosContainer.hpp
+++ b/query_execution/WorkOrderProtosContainer.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/WorkOrdersContainer.cpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrdersContainer.cpp b/query_execution/WorkOrdersContainer.cpp
index d72b6f3..a16a8e6 100644
--- a/query_execution/WorkOrdersContainer.cpp
+++ b/query_execution/WorkOrdersContainer.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/WorkOrdersContainer.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/WorkOrdersContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrdersContainer.hpp b/query_execution/WorkOrdersContainer.hpp
index d023daa..c1739bc 100644
--- a/query_execution/WorkOrdersContainer.hpp
+++ b/query_execution/WorkOrdersContainer.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_WORKORDERS_CONTAINER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index a582132..d497be6 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/Worker.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index 71d3ec9..44a7447 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_WORKER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/WorkerDirectory.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkerDirectory.hpp b/query_execution/WorkerDirectory.hpp
index a1f3b02..be5cfc1 100644
--- a/query_execution/WorkerDirectory.hpp
+++ b/query_execution/WorkerDirectory.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_WORKER_DIRECTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/WorkerMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkerMessage.hpp b/query_execution/WorkerMessage.hpp
index a0434de..bd84ac6 100644
--- a/query_execution/WorkerMessage.hpp
+++ b/query_execution/WorkerMessage.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_WORKER_MESSAGE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/WorkerSelectionPolicy.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkerSelectionPolicy.hpp b/query_execution/WorkerSelectionPolicy.hpp
index 289b1c4..4f4c6eb 100644
--- a/query_execution/WorkerSelectionPolicy.hpp
+++ b/query_execution/WorkerSelectionPolicy.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_WORKER_SELECTION_POLICY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/tests/BlockLocator_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/BlockLocator_unittest.cpp b/query_execution/tests/BlockLocator_unittest.cpp
index fd25e9e..465f2a3 100644
--- a/query_execution/tests/BlockLocator_unittest.cpp
+++ b/query_execution/tests/BlockLocator_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstdlib>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 050b315..f65ec53 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <climits>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/tests/WorkOrdersContainer_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkOrdersContainer_unittest.cpp b/query_execution/tests/WorkOrdersContainer_unittest.cpp
index cb583ab..a3b9ba1 100644
--- a/query_execution/tests/WorkOrdersContainer_unittest.cpp
+++ b/query_execution/tests/WorkOrdersContainer_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/tests/WorkerDirectory_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkerDirectory_unittest.cpp b/query_execution/tests/WorkerDirectory_unittest.cpp
index eb83583..2eab75d 100644
--- a/query_execution/tests/WorkerDirectory_unittest.cpp
+++ b/query_execution/tests/WorkerDirectory_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkerSelectionPolicy_unittest.cpp b/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
index 83d3c78..68de5e5 100644
--- a/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
+++ b/query_execution/tests/WorkerSelectionPolicy_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index de63bd0..c24ee89 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 if (ENABLE_DISTRIBUTED)
   set(QUICKSTEP_DISTRIBUTED TRUE)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 2ae6a4b..a4b538e 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/ExecutionGenerator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 9186707..d5c7b06 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXECUTION_GENERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index fc31c53..4fd7320 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/ExecutionHeuristics.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 92a7fe8..8ad3b7a 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXECUTION_HEURISTICS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/LogicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LogicalGenerator.cpp b/query_optimizer/LogicalGenerator.cpp
index 509a1a0..450c36a 100644
--- a/query_optimizer/LogicalGenerator.cpp
+++ b/query_optimizer/LogicalGenerator.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/LogicalGenerator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/LogicalGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LogicalGenerator.hpp b/query_optimizer/LogicalGenerator.hpp
index 43d813a..d4384c8 100644
--- a/query_optimizer/LogicalGenerator.hpp
+++ b/query_optimizer/LogicalGenerator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_GENERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/LogicalToPhysicalMapper.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LogicalToPhysicalMapper.hpp b/query_optimizer/LogicalToPhysicalMapper.hpp
index ba25455..869dad0 100644
--- a/query_optimizer/LogicalToPhysicalMapper.hpp
+++ b/query_optimizer/LogicalToPhysicalMapper.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_TO_PHYSICAL_MAPPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/Optimizer.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.cpp b/query_optimizer/Optimizer.cpp
index 08629fc..b77e430 100644
--- a/query_optimizer/Optimizer.cpp
+++ b/query_optimizer/Optimizer.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/Optimizer.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/Optimizer.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.hpp b/query_optimizer/Optimizer.hpp
index 18c927d..132b0b8 100644
--- a/query_optimizer/Optimizer.hpp
+++ b/query_optimizer/Optimizer.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/OptimizerContext.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerContext.cpp b/query_optimizer/OptimizerContext.cpp
index 4bdd24e..de7ba8c 100644
--- a/query_optimizer/OptimizerContext.cpp
+++ b/query_optimizer/OptimizerContext.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/OptimizerContext.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/OptimizerContext.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerContext.hpp b/query_optimizer/OptimizerContext.hpp
index bcab9d0..5be2a98 100644
--- a/query_optimizer/OptimizerContext.hpp
+++ b/query_optimizer/OptimizerContext.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_CONTEXT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/OptimizerTree.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerTree.hpp b/query_optimizer/OptimizerTree.hpp
index eb4de74..8ea2c6f 100644
--- a/query_optimizer/OptimizerTree.hpp
+++ b/query_optimizer/OptimizerTree.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_TREE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 897b212..8f19702 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/PhysicalGenerator.hpp"



[50/50] [abbrv] incubator-quickstep git commit: Only run the distributed execution related unit tests.

Posted by zu...@apache.org.
Only run the distributed execution related unit tests.


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

Branch: refs/heads/travis-grpc
Commit: 9c9f328698a38ef898b78167bae12d5b2a5a5f3f
Parents: cc39255
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Fri Aug 12 09:05:37 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Fri Aug 12 09:05:37 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9c9f3286/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 2650b6c..2e5e94e 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -73,7 +73,7 @@ script:
   - ./validate_cmakelists.py
   - ./cyclic_dependency.py
   - (cd build && make -j$MAKE_JOBS)
-  - (cd build && ctest -E native_net_client_message_bus_unittest --output-on-failure -j$TEST_JOBS)
+  - (cd build && ctest -R distributed --output-on-failure -j$TEST_JOBS)
 
 after_failure:
   - df -h


[40/50] [abbrv] incubator-quickstep git commit: Minor updates to Shiftboss.

Posted by zu...@apache.org.
Minor updates to Shiftboss.


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

Branch: refs/heads/travis-grpc
Commit: e443b2b8409a128cc5ba2bdf1a6d01ebf79e7e74
Parents: bd01748
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 10:33:10 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:33:10 2016 -0700

----------------------------------------------------------------------
 query_execution/QueryExecutionMessages.proto |  1 +
 query_execution/Shiftboss.cpp                | 89 ++++++++++++++++-------
 query_execution/Shiftboss.hpp                |  3 +
 3 files changed, 65 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f6a8b73..f680d35 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -87,6 +87,7 @@ message ShiftbossRegistrationMessage {
 }
 
 message ShiftbossRegistrationResponseMessage {
+  required uint64 shiftboss_index = 1;
 }
 
 message QueryInitiateMessage {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 120e8fb..24c91fe 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -56,6 +56,7 @@ using std::string;
 using std::unique_ptr;
 using std::vector;
 
+using tmb::MessageBus;
 using tmb::TaggedMessage;
 
 namespace quickstep {
@@ -78,6 +79,13 @@ void Shiftboss::run() {
     switch (annotated_message.tagged_message.message_type()) {
       case kShiftbossRegistrationResponseMessage: {
         foreman_client_id_ = annotated_message.sender;
+
+        const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+        serialization::ShiftbossRegistrationResponseMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        shiftboss_index_ = proto.shiftboss_index();
         break;
       }
       case kQueryInitiateMessage: {
@@ -117,10 +125,14 @@ void Shiftboss::run() {
                   << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
                   << "') from Foreman to worker " << worker_index;
 
-        QueryExecutionUtil::SendTMBMessage(bus_,
-                                           shiftboss_client_id_,
-                                           workers_->getClientID(worker_index),
-                                           move(worker_tagged_message));
+        const MessageBus::SendStatus send_status =
+            QueryExecutionUtil::SendTMBMessage(bus_,
+                                               shiftboss_client_id_,
+                                               workers_->getClientID(worker_index),
+                                               move(worker_tagged_message));
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
         break;
       }
       case kInitiateRebuildMessage: {
@@ -147,10 +159,14 @@ void Shiftboss::run() {
                   << "' message from worker (client " << annotated_message.sender << ") to Foreman";
 
         DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
-        QueryExecutionUtil::SendTMBMessage(bus_,
-                                           shiftboss_client_id_,
-                                           foreman_client_id_,
-                                           move(annotated_message.tagged_message));
+        const MessageBus::SendStatus send_status =
+            QueryExecutionUtil::SendTMBMessage(bus_,
+                                               shiftboss_client_id_,
+                                               foreman_client_id_,
+                                               move(annotated_message.tagged_message));
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to Foreman with TMB client ID " << foreman_client_id_;
         break;
       }
       case kSaveQueryResultMessage: {
@@ -178,10 +194,14 @@ void Shiftboss::run() {
         LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
                   << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
                   << "') to Foreman";
-        QueryExecutionUtil::SendTMBMessage(bus_,
-                                           shiftboss_client_id_,
-                                           foreman_client_id_,
-                                           move(message_response));
+        const MessageBus::SendStatus send_status =
+            QueryExecutionUtil::SendTMBMessage(bus_,
+                                               shiftboss_client_id_,
+                                               foreman_client_id_,
+                                               move(message_response));
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to Foreman with TMB client ID " << foreman_client_id_;
         break;
       }
       case kPoisonMessage: {
@@ -192,12 +212,14 @@ void Shiftboss::run() {
         tmb::MessageStyle broadcast_style;
         broadcast_style.Broadcast(true);
 
-        tmb::MessageBus::SendStatus send_status =
+        const MessageBus::SendStatus send_status =
             bus_->Send(shiftboss_client_id_,
                        worker_addresses_,
                        broadcast_style,
                        move(annotated_message.tagged_message));
-        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be broadcast from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to All workers";
         return;
       }
       default: {
@@ -280,10 +302,14 @@ void Shiftboss::processQueryInitiateMessage(
                                  kQueryInitiateResponseMessage);
   free(proto_bytes);
 
-  QueryExecutionUtil::SendTMBMessage(bus_,
-                                     shiftboss_client_id_,
-                                     foreman_client_id_,
-                                     move(message_response));
+  const MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         shiftboss_client_id_,
+                                         foreman_client_id_,
+                                         move(message_response));
+  CHECK(send_status == MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+      << " to Foreman with TMB client ID " << foreman_client_id_;
 }
 
 void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -307,8 +333,7 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   proto.set_query_id(query_id);
   proto.set_operator_index(op_index);
   proto.set_num_rebuild_work_orders(partially_filled_block_refs.size());
-  // TODO(zuyu): Multiple Shiftboss support.
-  proto.set_shiftboss_index(0);
+  proto.set_shiftboss_index(shiftboss_index_);
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -319,10 +344,14 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                                  kInitiateRebuildResponseMessage);
   free(proto_bytes);
 
-  QueryExecutionUtil::SendTMBMessage(bus_,
-                                     shiftboss_client_id_,
-                                     foreman_client_id_,
-                                     move(message_response));
+  const MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         shiftboss_client_id_,
+                                         foreman_client_id_,
+                                         move(message_response));
+  CHECK(send_status == MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+      << " to Foreman with TMB client ID " << foreman_client_id_;
 
   for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
     // NOTE(zuyu): Worker releases the memory after the execution of
@@ -347,10 +376,14 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
               << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
               << "') to worker " << worker_index;
 
-    QueryExecutionUtil::SendTMBMessage(bus_,
-                                       shiftboss_client_id_,
-                                       workers_->getClientID(worker_index),
-                                       move(worker_tagged_message));
+    const MessageBus::SendStatus send_status =
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           shiftboss_client_id_,
+                                           workers_->getClientID(worker_index),
+                                           move(worker_tagged_message));
+    CHECK(send_status == MessageBus::SendStatus::kOK)
+        << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+        << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 32d2408..9464a4d 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -223,6 +223,9 @@ class Shiftboss : public Thread {
 
   tmb::client_id shiftboss_client_id_, foreman_client_id_;
 
+  // Unique per Shiftboss instance.
+  std::uint64_t shiftboss_index_;
+
   // TMB recipients for all workers managed by this Shiftboss.
   tmb::Address worker_addresses_;
 


[21/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlockLayout.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.hpp b/storage/StorageBlockLayout.hpp
index 97a5704..9827a4c 100644
--- a/storage/StorageBlockLayout.hpp
+++ b/storage/StorageBlockLayout.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_BLOCK_LAYOUT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlockLayout.proto
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.proto b/storage/StorageBlockLayout.proto
index cafc0e9..6573470 100644
--- a/storage/StorageBlockLayout.proto
+++ b/storage/StorageBlockLayout.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//    University of Wisconsin-Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 
@@ -66,7 +66,7 @@ message IndexSubBlockDescription {
     SMA = 1;
     BLOOM_FILTER = 2;
     BITWEAVING_H = 3;
-    BITWEAVING_V = 4;    
+    BITWEAVING_V = 4;
   }
 
   required IndexSubBlockType sub_block_type = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageConfig.h.in
----------------------------------------------------------------------
diff --git a/storage/StorageConfig.h.in b/storage/StorageConfig.h.in
index eee1be9..4d9f978 100644
--- a/storage/StorageConfig.h.in
+++ b/storage/StorageConfig.h.in
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_HAVE_BITWEAVING

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageConstants.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageConstants.hpp b/storage/StorageConstants.hpp
index 0609cba..90028db 100644
--- a/storage/StorageConstants.hpp
+++ b/storage/StorageConstants.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_CONSTANTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageErrors.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageErrors.cpp b/storage/StorageErrors.cpp
index 2d38d9a..310ee4c 100644
--- a/storage/StorageErrors.cpp
+++ b/storage/StorageErrors.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/StorageErrors.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageErrors.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageErrors.hpp b/storage/StorageErrors.hpp
index d55867b..67a3e42 100644
--- a/storage/StorageErrors.hpp
+++ b/storage/StorageErrors.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_ERRORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index 8fc1224..56ca323 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 // This is included before other files so that we can conditionally determine

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index 50ddb0f..066953b 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageModule.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageModule.hpp b/storage/StorageModule.hpp
index 444f350..fd6b4e5 100644
--- a/storage/StorageModule.hpp
+++ b/storage/StorageModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Storage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SubBlockTypeRegistry.cpp
----------------------------------------------------------------------
diff --git a/storage/SubBlockTypeRegistry.cpp b/storage/SubBlockTypeRegistry.cpp
index f519c73..d35e422 100644
--- a/storage/SubBlockTypeRegistry.cpp
+++ b/storage/SubBlockTypeRegistry.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/SubBlockTypeRegistry.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SubBlockTypeRegistry.hpp
----------------------------------------------------------------------
diff --git a/storage/SubBlockTypeRegistry.hpp b/storage/SubBlockTypeRegistry.hpp
index e930fb4..c362d9f 100644
--- a/storage/SubBlockTypeRegistry.hpp
+++ b/storage/SubBlockTypeRegistry.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SUB_BLOCK_TYPE_REGISTRY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SubBlockTypeRegistryMacros.hpp
----------------------------------------------------------------------
diff --git a/storage/SubBlockTypeRegistryMacros.hpp b/storage/SubBlockTypeRegistryMacros.hpp
index 8a13bae..554e292 100644
--- a/storage/SubBlockTypeRegistryMacros.hpp
+++ b/storage/SubBlockTypeRegistryMacros.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SUB_BLOCK_TYPE_REGISTRY_MACROS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SubBlocksReference.hpp
----------------------------------------------------------------------
diff --git a/storage/SubBlocksReference.hpp b/storage/SubBlocksReference.hpp
index 8c0d6df..f0e240c 100644
--- a/storage/SubBlocksReference.hpp
+++ b/storage/SubBlocksReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SUB_BLOCKS_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/TupleIdSequence.hpp
----------------------------------------------------------------------
diff --git a/storage/TupleIdSequence.hpp b/storage/TupleIdSequence.hpp
index 096eb9d..5047270 100644
--- a/storage/TupleIdSequence.hpp
+++ b/storage/TupleIdSequence.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TUPLE_ID_SEQUENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/TupleReference.hpp
----------------------------------------------------------------------
diff --git a/storage/TupleReference.hpp b/storage/TupleReference.hpp
index 303282f..d53d852 100644
--- a/storage/TupleReference.hpp
+++ b/storage/TupleReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TUPLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/TupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/TupleStorageSubBlock.cpp b/storage/TupleStorageSubBlock.cpp
index aafcd33..c5d482f 100644
--- a/storage/TupleStorageSubBlock.cpp
+++ b/storage/TupleStorageSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/TupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/TupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/TupleStorageSubBlock.hpp b/storage/TupleStorageSubBlock.hpp
index 6da4698..aed6eea 100644
--- a/storage/TupleStorageSubBlock.hpp
+++ b/storage/TupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index e2a898e..70d4405 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/ValueAccessorUtil.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessorUtil.hpp b/storage/ValueAccessorUtil.hpp
index 8d0c315..8be3785 100644
--- a/storage/ValueAccessorUtil.hpp
+++ b/storage/ValueAccessorUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_VALUE_ACCESSOR_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/WindowAggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.cpp b/storage/WindowAggregationOperationState.cpp
index 49fa44d..58bdf18 100644
--- a/storage/WindowAggregationOperationState.cpp
+++ b/storage/WindowAggregationOperationState.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/WindowAggregationOperationState.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/WindowAggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.hpp b/storage/WindowAggregationOperationState.hpp
index 726b102..aaa8e37 100644
--- a/storage/WindowAggregationOperationState.hpp
+++ b/storage/WindowAggregationOperationState.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_WINDOW_AGGREGATION_OPERATION_STATE_HPP_
@@ -45,7 +45,7 @@ class WindowAggregateFunction;
  */
 
 /**
- * @brief Helper class for maintaining the state of window aggregation. 
+ * @brief Helper class for maintaining the state of window aggregation.
  **/
 class WindowAggregationOperationState {
  public:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/WindowAggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.proto b/storage/WindowAggregationOperationState.proto
index f879713..d05052e 100644
--- a/storage/WindowAggregationOperationState.proto
+++ b/storage/WindowAggregationOperationState.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//     University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/AggregationOperationState_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/AggregationOperationState_unittest.cpp b/storage/tests/AggregationOperationState_unittest.cpp
index 261e333..e1e5584 100644
--- a/storage/tests/AggregationOperationState_unittest.cpp
+++ b/storage/tests/AggregationOperationState_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
index df8b479..a511fcb 100644
--- a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/BloomFilterIndexSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/BloomFilterIndexSubBlock_unittest.cpp b/storage/tests/BloomFilterIndexSubBlock_unittest.cpp
index a9feae4..4f77c11 100644
--- a/storage/tests/BloomFilterIndexSubBlock_unittest.cpp
+++ b/storage/tests/BloomFilterIndexSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/CSBTreeIndexSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/CSBTreeIndexSubBlock_unittest.cpp b/storage/tests/CSBTreeIndexSubBlock_unittest.cpp
index 1815dfd..893156d 100644
--- a/storage/tests/CSBTreeIndexSubBlock_unittest.cpp
+++ b/storage/tests/CSBTreeIndexSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/CSBTreePrettyPrinter.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/CSBTreePrettyPrinter.cpp b/storage/tests/CSBTreePrettyPrinter.cpp
index c3ef131..f411f1f 100644
--- a/storage/tests/CSBTreePrettyPrinter.cpp
+++ b/storage/tests/CSBTreePrettyPrinter.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/tests/CSBTreePrettyPrinter.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/CSBTreePrettyPrinter.hpp
----------------------------------------------------------------------
diff --git a/storage/tests/CSBTreePrettyPrinter.hpp b/storage/tests/CSBTreePrettyPrinter.hpp
index e36e7e6..accc3ea 100644
--- a/storage/tests/CSBTreePrettyPrinter.hpp
+++ b/storage/tests/CSBTreePrettyPrinter.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TESTS_CSBTREE_PRETTY_PRINTER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/CompressedColumnStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/CompressedColumnStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/CompressedColumnStoreTupleStorageSubBlock_unittest.cpp
index a7c491e..8a8f5ef 100644
--- a/storage/tests/CompressedColumnStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/CompressedColumnStoreTupleStorageSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
index 5bf8567..530507b 100644
--- a/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/DataExchange_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/DataExchange_unittest.cpp b/storage/tests/DataExchange_unittest.cpp
index 4bad17b..9c75150 100644
--- a/storage/tests/DataExchange_unittest.cpp
+++ b/storage/tests/DataExchange_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstdlib>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/EvictionPolicy_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/EvictionPolicy_unittest.cpp b/storage/tests/EvictionPolicy_unittest.cpp
index 674de32..e7fef3d 100644
--- a/storage/tests/EvictionPolicy_unittest.cpp
+++ b/storage/tests/EvictionPolicy_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/FileManagerHdfs_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/FileManagerHdfs_unittest.cpp b/storage/tests/FileManagerHdfs_unittest.cpp
index cd59160..1ef2bd4 100644
--- a/storage/tests/FileManagerHdfs_unittest.cpp
+++ b/storage/tests/FileManagerHdfs_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "gtest/gtest.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/FileManagerLocal_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/FileManagerLocal_unittest.cpp b/storage/tests/FileManagerLocal_unittest.cpp
index a2e7139..282f041 100644
--- a/storage/tests/FileManagerLocal_unittest.cpp
+++ b/storage/tests/FileManagerLocal_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "gtest/gtest.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/FileManager_unittest_common.hpp
----------------------------------------------------------------------
diff --git a/storage/tests/FileManager_unittest_common.hpp b/storage/tests/FileManager_unittest_common.hpp
index 6ea87dd..06d64ec 100644
--- a/storage/tests/FileManager_unittest_common.hpp
+++ b/storage/tests/FileManager_unittest_common.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TESTS_FILEMANAGER_UNITTEST_COMMON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/HashTable_unittest_common.hpp
----------------------------------------------------------------------
diff --git a/storage/tests/HashTable_unittest_common.hpp b/storage/tests/HashTable_unittest_common.hpp
index c1df6d1..541706f 100644
--- a/storage/tests/HashTable_unittest_common.hpp
+++ b/storage/tests/HashTable_unittest_common.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TESTS_HASHTABLE_UNITTEST_COMMON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/LinearOpenAddressingHashTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/LinearOpenAddressingHashTable_unittest.cpp b/storage/tests/LinearOpenAddressingHashTable_unittest.cpp
index 9d8b89a..d06bd73 100644
--- a/storage/tests/LinearOpenAddressingHashTable_unittest.cpp
+++ b/storage/tests/LinearOpenAddressingHashTable_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "gtest/gtest.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/MockTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/tests/MockTupleStorageSubBlock.hpp b/storage/tests/MockTupleStorageSubBlock.hpp
index d55cc07..0c92072 100644
--- a/storage/tests/MockTupleStorageSubBlock.hpp
+++ b/storage/tests/MockTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TESTS_MOCK_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
index d4ab0b6..304fa07 100644
--- a/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstring>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/SMAIndexSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/SMAIndexSubBlock_unittest.cpp b/storage/tests/SMAIndexSubBlock_unittest.cpp
index ad842c2..0948be1 100644
--- a/storage/tests/SMAIndexSubBlock_unittest.cpp
+++ b/storage/tests/SMAIndexSubBlock_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/SeparateChainingHashTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/SeparateChainingHashTable_unittest.cpp b/storage/tests/SeparateChainingHashTable_unittest.cpp
index 3ecd818..21b0300 100644
--- a/storage/tests/SeparateChainingHashTable_unittest.cpp
+++ b/storage/tests/SeparateChainingHashTable_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "gtest/gtest.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/SimpleScalarSeparateChainingHashTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/SimpleScalarSeparateChainingHashTable_unittest.cpp b/storage/tests/SimpleScalarSeparateChainingHashTable_unittest.cpp
index 3199663..b1d793c 100644
--- a/storage/tests/SimpleScalarSeparateChainingHashTable_unittest.cpp
+++ b/storage/tests/SimpleScalarSeparateChainingHashTable_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "gtest/gtest.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/SplitRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/SplitRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/SplitRowStoreTupleStorageSubBlock_unittest.cpp
index 43a85f7..2943343 100644
--- a/storage/tests/SplitRowStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/SplitRowStoreTupleStorageSubBlock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/StorageBlockSort_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/StorageBlockSort_unittest.cpp b/storage/tests/StorageBlockSort_unittest.cpp
index e4e31e2..559f6c3 100644
--- a/storage/tests/StorageBlockSort_unittest.cpp
+++ b/storage/tests/StorageBlockSort_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/StorageManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/StorageManager_unittest.cpp b/storage/tests/StorageManager_unittest.cpp
index 4c252a1..6245240 100644
--- a/storage/tests/StorageManager_unittest.cpp
+++ b/storage/tests/StorageManager_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <atomic>



[28/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Logical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Logical.hpp b/query_optimizer/logical/Logical.hpp
index a1e9155..6396f15 100644
--- a/query_optimizer/logical/Logical.hpp
+++ b/query_optimizer/logical/Logical.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_LOGICAL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/LogicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/LogicalType.hpp b/query_optimizer/logical/LogicalType.hpp
index c82fb47..d1011b0 100644
--- a/query_optimizer/logical/LogicalType.hpp
+++ b/query_optimizer/logical/LogicalType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_LOGICAL_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/MultiwayCartesianJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/MultiwayCartesianJoin.cpp b/query_optimizer/logical/MultiwayCartesianJoin.cpp
index e672f7f..60a0e22 100644
--- a/query_optimizer/logical/MultiwayCartesianJoin.cpp
+++ b/query_optimizer/logical/MultiwayCartesianJoin.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/MultiwayCartesianJoin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/MultiwayCartesianJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/MultiwayCartesianJoin.hpp b/query_optimizer/logical/MultiwayCartesianJoin.hpp
index 8eb0cd7..6f9121d 100644
--- a/query_optimizer/logical/MultiwayCartesianJoin.hpp
+++ b/query_optimizer/logical/MultiwayCartesianJoin.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_MULTIWAY_CARTESIAN_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/NestedLoopsJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/NestedLoopsJoin.hpp b/query_optimizer/logical/NestedLoopsJoin.hpp
index febd845..5950e1d 100644
--- a/query_optimizer/logical/NestedLoopsJoin.hpp
+++ b/query_optimizer/logical/NestedLoopsJoin.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_NESTED_LOOPS_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/OptimizerLogicalModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/OptimizerLogicalModule.hpp b/query_optimizer/logical/OptimizerLogicalModule.hpp
index cc8d35f..fe05718 100644
--- a/query_optimizer/logical/OptimizerLogicalModule.hpp
+++ b/query_optimizer/logical/OptimizerLogicalModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup OptimizerLogical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/PatternMatcher.hpp b/query_optimizer/logical/PatternMatcher.hpp
index de8609e..b931ace 100644
--- a/query_optimizer/logical/PatternMatcher.hpp
+++ b/query_optimizer/logical/PatternMatcher.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_PATTERN_MATCHER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Project.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Project.cpp b/query_optimizer/logical/Project.cpp
index fd9200e..18c8fe9 100644
--- a/query_optimizer/logical/Project.cpp
+++ b/query_optimizer/logical/Project.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/Project.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Project.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Project.hpp b/query_optimizer/logical/Project.hpp
index 02113fa..83b352c 100644
--- a/query_optimizer/logical/Project.hpp
+++ b/query_optimizer/logical/Project.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_PROJECT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Sample.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Sample.cpp b/query_optimizer/logical/Sample.cpp
index 95829d7..577292c 100644
--- a/query_optimizer/logical/Sample.cpp
+++ b/query_optimizer/logical/Sample.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/Sample.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Sample.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Sample.hpp b/query_optimizer/logical/Sample.hpp
index 9ffe93f..e2403c5 100644
--- a/query_optimizer/logical/Sample.hpp
+++ b/query_optimizer/logical/Sample.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_SAMPLE_HPP_
@@ -41,7 +43,7 @@ class Sample;
 typedef std::shared_ptr<const Sample> SamplePtr;
 
 /**
- * @brief Sample operator that samples the data at a block level or 
+ * @brief Sample operator that samples the data at a block level or
  *        at the tuple level and outputs them as a new relation.
  */
 class Sample : public Logical {
@@ -87,7 +89,7 @@ class Sample : public Logical {
    *        relation produced by \p input.
    * @param input The input operator to this Sample.
    * @param is_block_sample Flag indicating if the sampling is either a block or
-   *                        tuple level sampling 
+   *                        tuple level sampling
    * @param percentage The percentage of tuples/blocks to be sampled
    *
    * @return An immutable Sample.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/SharedSubplanReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/SharedSubplanReference.cpp b/query_optimizer/logical/SharedSubplanReference.cpp
index b5d3143..3a0fcb3 100644
--- a/query_optimizer/logical/SharedSubplanReference.cpp
+++ b/query_optimizer/logical/SharedSubplanReference.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/SharedSubplanReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/SharedSubplanReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/SharedSubplanReference.hpp b/query_optimizer/logical/SharedSubplanReference.hpp
index 8f0e37b..79ef4ad 100644
--- a/query_optimizer/logical/SharedSubplanReference.hpp
+++ b/query_optimizer/logical/SharedSubplanReference.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_SHARED_SUBPLAN_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Sort.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Sort.cpp b/query_optimizer/logical/Sort.cpp
index 2719491..3667f64 100644
--- a/query_optimizer/logical/Sort.cpp
+++ b/query_optimizer/logical/Sort.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/Sort.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Sort.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Sort.hpp b/query_optimizer/logical/Sort.hpp
index 2233151..894970b 100644
--- a/query_optimizer/logical/Sort.hpp
+++ b/query_optimizer/logical/Sort.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_SORT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/TableGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TableGenerator.hpp b/query_optimizer/logical/TableGenerator.hpp
index ea8826a..15ca985 100644
--- a/query_optimizer/logical/TableGenerator.hpp
+++ b/query_optimizer/logical/TableGenerator.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_TABLE_GENERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/TableReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TableReference.cpp b/query_optimizer/logical/TableReference.cpp
index 8f0362a..6d94665 100644
--- a/query_optimizer/logical/TableReference.cpp
+++ b/query_optimizer/logical/TableReference.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/TableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/TableReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TableReference.hpp b/query_optimizer/logical/TableReference.hpp
index e48368d..6788c82 100644
--- a/query_optimizer/logical/TableReference.hpp
+++ b/query_optimizer/logical/TableReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_LOGICAL_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/TopLevelPlan.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TopLevelPlan.cpp b/query_optimizer/logical/TopLevelPlan.cpp
index 8b2cdde..3a6eb44 100644
--- a/query_optimizer/logical/TopLevelPlan.cpp
+++ b/query_optimizer/logical/TopLevelPlan.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/TopLevelPlan.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/TopLevelPlan.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/TopLevelPlan.hpp b/query_optimizer/logical/TopLevelPlan.hpp
index 747a14e..0b8f1a4 100644
--- a/query_optimizer/logical/TopLevelPlan.hpp
+++ b/query_optimizer/logical/TopLevelPlan.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_TOP_LEVEL_PLAN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/UpdateTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/UpdateTable.cpp b/query_optimizer/logical/UpdateTable.cpp
index e34d8ce..8f02f5d 100644
--- a/query_optimizer/logical/UpdateTable.cpp
+++ b/query_optimizer/logical/UpdateTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/UpdateTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/UpdateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/UpdateTable.hpp b/query_optimizer/logical/UpdateTable.hpp
index 6261e25..ec0d50f 100644
--- a/query_optimizer/logical/UpdateTable.hpp
+++ b/query_optimizer/logical/UpdateTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_UPDATETABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/WindowAggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/WindowAggregate.cpp b/query_optimizer/logical/WindowAggregate.cpp
index 0d747b6..cc09d97 100644
--- a/query_optimizer/logical/WindowAggregate.cpp
+++ b/query_optimizer/logical/WindowAggregate.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/WindowAggregate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/WindowAggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/WindowAggregate.hpp b/query_optimizer/logical/WindowAggregate.hpp
index dcd9a7d..b156938 100644
--- a/query_optimizer/logical/WindowAggregate.hpp
+++ b/query_optimizer/logical/WindowAggregate.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_WINDOW_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.cpp b/query_optimizer/physical/Aggregate.cpp
index c582bba..35476af 100644
--- a/query_optimizer/physical/Aggregate.cpp
+++ b/query_optimizer/physical/Aggregate.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/Aggregate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.hpp b/query_optimizer/physical/Aggregate.hpp
index 2c2aee7..de36cc3 100644
--- a/query_optimizer/physical/Aggregate.hpp
+++ b/query_optimizer/physical/Aggregate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/BinaryJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/BinaryJoin.cpp b/query_optimizer/physical/BinaryJoin.cpp
index 1689331..30e2e8d 100644
--- a/query_optimizer/physical/BinaryJoin.cpp
+++ b/query_optimizer/physical/BinaryJoin.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/BinaryJoin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/BinaryJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/BinaryJoin.hpp b/query_optimizer/physical/BinaryJoin.hpp
index 4a11b9f..4b5671d 100644
--- a/query_optimizer/physical/BinaryJoin.hpp
+++ b/query_optimizer/physical/BinaryJoin.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_BINARY_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index 1ad30e4..3b7d3f0 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_queryoptimizer_physical_Aggregate Aggregate.cpp Aggregate.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CopyFrom.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CopyFrom.cpp b/query_optimizer/physical/CopyFrom.cpp
index b9cd06b..8448d4e 100644
--- a/query_optimizer/physical/CopyFrom.cpp
+++ b/query_optimizer/physical/CopyFrom.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/CopyFrom.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CopyFrom.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CopyFrom.hpp b/query_optimizer/physical/CopyFrom.hpp
index 8c13ca5..ecbf318 100644
--- a/query_optimizer/physical/CopyFrom.hpp
+++ b/query_optimizer/physical/CopyFrom.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_COPYFROM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CreateIndex.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateIndex.cpp b/query_optimizer/physical/CreateIndex.cpp
index e9489fa..af7d500 100644
--- a/query_optimizer/physical/CreateIndex.cpp
+++ b/query_optimizer/physical/CreateIndex.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/CreateIndex.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CreateIndex.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateIndex.hpp b/query_optimizer/physical/CreateIndex.hpp
index b66c1d8..3847736 100644
--- a/query_optimizer/physical/CreateIndex.hpp
+++ b/query_optimizer/physical/CreateIndex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_CREATEINDEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CreateTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateTable.cpp b/query_optimizer/physical/CreateTable.cpp
index d7c3732..d42eac3 100644
--- a/query_optimizer/physical/CreateTable.cpp
+++ b/query_optimizer/physical/CreateTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/CreateTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/CreateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateTable.hpp b/query_optimizer/physical/CreateTable.hpp
index 181ff2b..8e3bbd4 100644
--- a/query_optimizer/physical/CreateTable.hpp
+++ b/query_optimizer/physical/CreateTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_CREATETABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/DeleteTuples.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/DeleteTuples.cpp b/query_optimizer/physical/DeleteTuples.cpp
index ccce872..2c93502 100644
--- a/query_optimizer/physical/DeleteTuples.cpp
+++ b/query_optimizer/physical/DeleteTuples.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/DeleteTuples.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/DeleteTuples.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/DeleteTuples.hpp b/query_optimizer/physical/DeleteTuples.hpp
index fed0a1d..bd1e847 100644
--- a/query_optimizer/physical/DeleteTuples.hpp
+++ b/query_optimizer/physical/DeleteTuples.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_DELETE_TUPLES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/DropTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/DropTable.cpp b/query_optimizer/physical/DropTable.cpp
index ab3ffd3..6a4b5bd 100644
--- a/query_optimizer/physical/DropTable.cpp
+++ b/query_optimizer/physical/DropTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/DropTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/DropTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/DropTable.hpp b/query_optimizer/physical/DropTable.hpp
index 9884f0b..ef6020b 100644
--- a/query_optimizer/physical/DropTable.hpp
+++ b/query_optimizer/physical/DropTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_DROPTABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/HashJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.cpp b/query_optimizer/physical/HashJoin.cpp
index 71c3692..e186072 100644
--- a/query_optimizer/physical/HashJoin.cpp
+++ b/query_optimizer/physical/HashJoin.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/HashJoin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index 988b139..c513f77 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_HASHJOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/InsertSelection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertSelection.cpp b/query_optimizer/physical/InsertSelection.cpp
index 0e45222..42a888f 100644
--- a/query_optimizer/physical/InsertSelection.cpp
+++ b/query_optimizer/physical/InsertSelection.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/InsertSelection.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/InsertSelection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertSelection.hpp b/query_optimizer/physical/InsertSelection.hpp
index 8bdcfcb..4e22a6e 100644
--- a/query_optimizer/physical/InsertSelection.hpp
+++ b/query_optimizer/physical/InsertSelection.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_INSERT_SELECTION_HPP_


[47/50] [abbrv] incubator-quickstep git commit: Fixed a bug in deserializing WindowAggrWorkOrder.

Posted by zu...@apache.org.
Fixed a bug in deserializing WindowAggrWorkOrder.


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

Branch: refs/heads/travis-grpc
Commit: 6ee9842fbebd8dac38c35c383bad018b1b7fcb09
Parents: d9135a8
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Fri Aug 12 08:59:12 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Fri Aug 12 08:59:12 2016 -0700

----------------------------------------------------------------------
 relational_operators/WorkOrderFactory.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6ee9842f/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 7d7af59..6970486 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -434,7 +434,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)),
           move(blocks),
           query_context->getInsertDestination(
-              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
+              proto.GetExtension(serialization::WindowAggregationWorkOrder::insert_destination_index)));
     }
     default:
       LOG(FATAL) << "Unknown WorkOrder Type in WorkOrderFactory::ReconstructFromProto";


[48/50] [abbrv] incubator-quickstep git commit: CI w/ gRPC to build the distributed version.

Posted by zu...@apache.org.
CI w/ gRPC to build the distributed version.


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

Branch: refs/heads/travis-grpc
Commit: 9ec7c4bcfc21f45c1d428416fe0eb9390c673f72
Parents: 6ee9842
Author: Zuyu Zhang <zz...@pivotal.io>
Authored: Mon May 30 11:57:22 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Fri Aug 12 09:04:15 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 26 ++++++++++++++++++--------
 1 file changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ec7c4bc/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 54a0c8a..3c00cd7 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -1,4 +1,4 @@
-# NOTE(quickstep-team): In Travis-CI, jobs timeout if they take more than 120
+# NOTE(quickstep-team): In Travis-CI, jobs timeout if they take more than 50
 # mins or if there is no log output for more than 10 mins. Hence, we use -O0 to
 # speed up compilation in release build. Also, jobs can only use upto 20GB of
 # disk space. Hence, we minimize the amount of debug symbol using -g0 (DEBUG
@@ -15,13 +15,13 @@ compiler:
   - clang
 
 env:
-  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
-  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=selection
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=selection
+  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=none
+  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=none
 
 install:
-  - if [ "$CC" = "gcc" ]; then
+  - if [ "$VECTOR_COPY_ELISION_LEVEL" = "selection" ] && [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;
     else
       export MAKE_JOBS=2;
@@ -38,11 +38,22 @@ install:
   - export DEBUG_FLAGS="-g0";
   - export RELEASE_FLAGS="-O0 -DNDEBUG";
   - export LINKER_FLAGS="-s"
+  # Protobuf 3 beta.
+  - wget https://github.com/google/protobuf/releases/download/v3.0.0-beta-3/protobuf-cpp-3.0.0-beta-3.tar.gz
+  - tar -xzvf protobuf-cpp-3.0.0-beta-3.tar.gz
+  - pushd protobuf-3.0.0-beta-3 && ./configure --prefix=/usr && make && sudo make install && popd
+  - rm -rf protobuf*
+  - export PROTOC=`which protoc`
+  # gRPC.
+  - git clone https://github.com/grpc/grpc.git
+  - pushd grpc && git checkout release-0_14_1 && git submodule update --init && make && sudo make install && popd
+  - rm -rf grpc
 
 before_script:
   - $CC --version
   - $CXX --version
   - $CLINKER --version
+  - $PROTOC --version
   - (cd build &&
      cmake -D CMAKE_BUILD_TYPE=$BUILD_TYPE
            -D BUILD_SHARED_LIBS=On
@@ -55,14 +66,15 @@ before_script:
            -D CMAKE_CXX_COMPILER=$CXX
            -D CMAKE_LINKER=$CLINKER
            -D USE_TCMALLOC=0
-           -D VECTOR_COPY_ELISION_LEVEL=$VECTOR_COPY_ELISION_LEVEL ..)
+           -D VECTOR_COPY_ELISION_LEVEL=$VECTOR_COPY_ELISION_LEVEL
+           -D ENABLE_DISTRIBUTED=ON ..)
 
 script:
   - ./third_party/cpplint/lint_everything.py
   - ./validate_cmakelists.py
   - ./cyclic_dependency.py
   - (cd build && make -j$MAKE_JOBS)
-  - (cd build && ctest --output-on-failure -j$TEST_JOBS)
+  - (cd build && ctest -E native_net_client_message_bus_unittest --output-on-failure -j$TEST_JOBS)
 
 after_failure:
   - df -h
@@ -84,8 +96,6 @@ addons:
       - g++-5
       - clang-3.7
       - binutils-gold
-      - libprotobuf-dev
-      - protobuf-compiler
       - libgtest-dev
       - python-networkx
       - libnuma-dev


[17/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/memory_mirror_message_bus_with_zookeeper_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/memory_mirror_message_bus_with_zookeeper_unittest.cc b/third_party/tmb/tests/memory_mirror_message_bus_with_zookeeper_unittest.cc
index 801c530..69d1327 100644
--- a/third_party/tmb/tests/memory_mirror_message_bus_with_zookeeper_unittest.cc
+++ b/third_party/tmb/tests/memory_mirror_message_bus_with_zookeeper_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/message_bus_unittest_common.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/message_bus_unittest_common.h b/third_party/tmb/tests/message_bus_unittest_common.h
index b47f7ae..e0994da 100644
--- a/third_party/tmb/tests/message_bus_unittest_common.h
+++ b/third_party/tmb/tests/message_bus_unittest_common.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TESTS_MESSAGE_BUS_UNITTEST_COMMON_H_
 #define TESTS_MESSAGE_BUS_UNITTEST_COMMON_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/native_logging_message_bus_async_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/native_logging_message_bus_async_unittest.cc b/third_party/tmb/tests/native_logging_message_bus_async_unittest.cc
index a6be8e9..62b4aeb 100644
--- a/third_party/tmb/tests/native_logging_message_bus_async_unittest.cc
+++ b/third_party/tmb/tests/native_logging_message_bus_async_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/native_logging_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/native_logging_message_bus_unittest.cc b/third_party/tmb/tests/native_logging_message_bus_unittest.cc
index c867fca..837c072 100644
--- a/third_party/tmb/tests/native_logging_message_bus_unittest.cc
+++ b/third_party/tmb/tests/native_logging_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/native_net_client_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/native_net_client_message_bus_unittest.cc b/third_party/tmb/tests/native_net_client_message_bus_unittest.cc
index 8c7d2eb..a484cf3 100644
--- a/third_party/tmb/tests/native_net_client_message_bus_unittest.cc
+++ b/third_party/tmb/tests/native_net_client_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/pure_memory_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/pure_memory_message_bus_unittest.cc b/third_party/tmb/tests/pure_memory_message_bus_unittest.cc
index c8d5f2e..e947002 100644
--- a/third_party/tmb/tests/pure_memory_message_bus_unittest.cc
+++ b/third_party/tmb/tests/pure_memory_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/rcu_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/rcu_unittest.cc b/third_party/tmb/tests/rcu_unittest.cc
index fee8061..fb71eef 100644
--- a/third_party/tmb/tests/rcu_unittest.cc
+++ b/third_party/tmb/tests/rcu_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 <memory>
 #include <string>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/sqlite_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/sqlite_message_bus_unittest.cc b/third_party/tmb/tests/sqlite_message_bus_unittest.cc
index c1c8367..719772b 100644
--- a/third_party/tmb/tests/sqlite_message_bus_unittest.cc
+++ b/third_party/tmb/tests/sqlite_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/voltdb_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/voltdb_message_bus_unittest.cc b/third_party/tmb/tests/voltdb_message_bus_unittest.cc
index 9e025fa..968691e 100644
--- a/third_party/tmb/tests/voltdb_message_bus_unittest.cc
+++ b/third_party/tmb/tests/voltdb_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/zookeeper_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/zookeeper_message_bus_unittest.cc b/third_party/tmb/tests/zookeeper_message_bus_unittest.cc
index 2e4f086..14cfc64 100644
--- a/third_party/tmb/tests/zookeeper_message_bus_unittest.cc
+++ b/third_party/tmb/tests/zookeeper_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/threading/CMakeLists.txt b/threading/CMakeLists.txt
index 70c1f79..1a9f33e 100644
--- a/threading/CMakeLists.txt
+++ b/threading/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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(CheckIncludeFileCXX)
 include(CheckCXXSourceCompiles)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/ConditionVariable.cpp
----------------------------------------------------------------------
diff --git a/threading/ConditionVariable.cpp b/threading/ConditionVariable.cpp
index 56f8ab8..e9799a6 100644
--- a/threading/ConditionVariable.cpp
+++ b/threading/ConditionVariable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "threading/ConditionVariable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/ConditionVariable.hpp
----------------------------------------------------------------------
diff --git a/threading/ConditionVariable.hpp b/threading/ConditionVariable.hpp
index 2fe2e45..dc63be3 100644
--- a/threading/ConditionVariable.hpp
+++ b/threading/ConditionVariable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CONDITION_VARIABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/Mutex.cpp
----------------------------------------------------------------------
diff --git a/threading/Mutex.cpp b/threading/Mutex.cpp
index ec8caef..4e99877 100644
--- a/threading/Mutex.cpp
+++ b/threading/Mutex.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "threading/Mutex.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/Mutex.hpp
----------------------------------------------------------------------
diff --git a/threading/Mutex.hpp b/threading/Mutex.hpp
index 7107b7c..c5af74d 100644
--- a/threading/Mutex.hpp
+++ b/threading/Mutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/SharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/SharedMutex.hpp b/threading/SharedMutex.hpp
index f760e25..dd46e80 100644
--- a/threading/SharedMutex.hpp
+++ b/threading/SharedMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/SpinMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/SpinMutex.hpp b/threading/SpinMutex.hpp
index b64a6ca..5ed1405 100644
--- a/threading/SpinMutex.hpp
+++ b/threading/SpinMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_SPIN_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/SpinSharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/SpinSharedMutex.hpp b/threading/SpinSharedMutex.hpp
index d7bc2b3..8b1060b 100644
--- a/threading/SpinSharedMutex.hpp
+++ b/threading/SpinSharedMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_SPIN_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/Thread.cpp
----------------------------------------------------------------------
diff --git a/threading/Thread.cpp b/threading/Thread.cpp
index 3ab31c7..d0a2111 100644
--- a/threading/Thread.cpp
+++ b/threading/Thread.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "threading/Thread.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/Thread.hpp
----------------------------------------------------------------------
diff --git a/threading/Thread.hpp b/threading/Thread.hpp
index ebe1f2c..1247964 100644
--- a/threading/Thread.hpp
+++ b/threading/Thread.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_THREAD_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/ThreadIDBasedMap.hpp
----------------------------------------------------------------------
diff --git a/threading/ThreadIDBasedMap.hpp b/threading/ThreadIDBasedMap.hpp
index 8f2af71..958615c 100644
--- a/threading/ThreadIDBasedMap.hpp
+++ b/threading/ThreadIDBasedMap.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_THREAD_ID_BASED_MAP_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/ThreadUtil.hpp
----------------------------------------------------------------------
diff --git a/threading/ThreadUtil.hpp b/threading/ThreadUtil.hpp
index 3dbf462..d4c189c 100644
--- a/threading/ThreadUtil.hpp
+++ b/threading/ThreadUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_THREAD_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/ThreadingConfig.h.in
----------------------------------------------------------------------
diff --git a/threading/ThreadingConfig.h.in b/threading/ThreadingConfig.h.in
index 2b4b89b..238381a 100644
--- a/threading/ThreadingConfig.h.in
+++ b/threading/ThreadingConfig.h.in
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_HAVE_CPP11_THREADS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/ThreadingModule.hpp
----------------------------------------------------------------------
diff --git a/threading/ThreadingModule.hpp b/threading/ThreadingModule.hpp
index a1cf419..7cc7cf5 100644
--- a/threading/ThreadingModule.hpp
+++ b/threading/ThreadingModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Threading

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/WinThreadsAPI.hpp
----------------------------------------------------------------------
diff --git a/threading/WinThreadsAPI.hpp b/threading/WinThreadsAPI.hpp
index 7af58e8..36b844a 100644
--- a/threading/WinThreadsAPI.hpp
+++ b/threading/WinThreadsAPI.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_WIN_THREADS_API_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/cpp11/ConditionVariable.hpp
----------------------------------------------------------------------
diff --git a/threading/cpp11/ConditionVariable.hpp b/threading/cpp11/ConditionVariable.hpp
index cbf258c..2aea582 100644
--- a/threading/cpp11/ConditionVariable.hpp
+++ b/threading/cpp11/ConditionVariable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CPP11_CONDITION_VARIABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/cpp11/Mutex.hpp
----------------------------------------------------------------------
diff --git a/threading/cpp11/Mutex.hpp b/threading/cpp11/Mutex.hpp
index 503d71b..22ae975 100644
--- a/threading/cpp11/Mutex.hpp
+++ b/threading/cpp11/Mutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CPP11_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/cpp11/SharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/cpp11/SharedMutex.hpp b/threading/cpp11/SharedMutex.hpp
index dc378de..e34aede 100644
--- a/threading/cpp11/SharedMutex.hpp
+++ b/threading/cpp11/SharedMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CPP11_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/cpp11/Thread.hpp
----------------------------------------------------------------------
diff --git a/threading/cpp11/Thread.hpp b/threading/cpp11/Thread.hpp
index f67145e..326a26d 100644
--- a/threading/cpp11/Thread.hpp
+++ b/threading/cpp11/Thread.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CPP11_THREAD_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/cpp11/cpp14/SharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/cpp11/cpp14/SharedMutex.hpp b/threading/cpp11/cpp14/SharedMutex.hpp
index 9b8ff3c..9fc58d2 100644
--- a/threading/cpp11/cpp14/SharedMutex.hpp
+++ b/threading/cpp11/cpp14/SharedMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CPP11_CPP14_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/cpp11/cpp17/SharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/cpp11/cpp17/SharedMutex.hpp b/threading/cpp11/cpp17/SharedMutex.hpp
index c9bca9c..f1c7d6c 100644
--- a/threading/cpp11/cpp17/SharedMutex.hpp
+++ b/threading/cpp11/cpp17/SharedMutex.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_CPP11_CPP17_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/posix/ConditionVariable.hpp
----------------------------------------------------------------------
diff --git a/threading/posix/ConditionVariable.hpp b/threading/posix/ConditionVariable.hpp
index b467c45..5d5d28c 100644
--- a/threading/posix/ConditionVariable.hpp
+++ b/threading/posix/ConditionVariable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_POSIX_CONDITION_VARIABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/posix/Mutex.hpp
----------------------------------------------------------------------
diff --git a/threading/posix/Mutex.hpp b/threading/posix/Mutex.hpp
index 4d48522..77b7b14 100644
--- a/threading/posix/Mutex.hpp
+++ b/threading/posix/Mutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_POSIX_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/posix/SharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/posix/SharedMutex.hpp b/threading/posix/SharedMutex.hpp
index a7b40ab..d53897a 100644
--- a/threading/posix/SharedMutex.hpp
+++ b/threading/posix/SharedMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_POSIX_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/posix/Thread.hpp
----------------------------------------------------------------------
diff --git a/threading/posix/Thread.hpp b/threading/posix/Thread.hpp
index a3b22d9..9968d59 100644
--- a/threading/posix/Thread.hpp
+++ b/threading/posix/Thread.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_POSIX_THREAD_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/tests/Mutex_unittest.cpp
----------------------------------------------------------------------
diff --git a/threading/tests/Mutex_unittest.cpp b/threading/tests/Mutex_unittest.cpp
index 66e6d53..0e15f52 100644
--- a/threading/tests/Mutex_unittest.cpp
+++ b/threading/tests/Mutex_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <chrono>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/tests/SharedMutex_unittest.cpp
----------------------------------------------------------------------
diff --git a/threading/tests/SharedMutex_unittest.cpp b/threading/tests/SharedMutex_unittest.cpp
index ad92692..4941f25 100644
--- a/threading/tests/SharedMutex_unittest.cpp
+++ b/threading/tests/SharedMutex_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <atomic>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/windows/ConditionVariable.hpp
----------------------------------------------------------------------
diff --git a/threading/windows/ConditionVariable.hpp b/threading/windows/ConditionVariable.hpp
index b7dc742..881db77 100644
--- a/threading/windows/ConditionVariable.hpp
+++ b/threading/windows/ConditionVariable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_WINDOWS_CONDITION_VARIABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/windows/Mutex.hpp
----------------------------------------------------------------------
diff --git a/threading/windows/Mutex.hpp b/threading/windows/Mutex.hpp
index 6400c01..ec78302 100644
--- a/threading/windows/Mutex.hpp
+++ b/threading/windows/Mutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_WINDOWS_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/windows/SharedMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/windows/SharedMutex.hpp b/threading/windows/SharedMutex.hpp
index d330de6..b571a9a 100644
--- a/threading/windows/SharedMutex.hpp
+++ b/threading/windows/SharedMutex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_WINDOWS_SHARED_MUTEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/threading/windows/Thread.hpp
----------------------------------------------------------------------
diff --git a/threading/windows/Thread.hpp b/threading/windows/Thread.hpp
index ff6a787..55f4ae3 100644
--- a/threading/windows/Thread.hpp
+++ b/threading/windows/Thread.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_THREADING_WINDOWS_THREAD_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/AccessMode.cpp
----------------------------------------------------------------------
diff --git a/transaction/AccessMode.cpp b/transaction/AccessMode.cpp
index 3618bfd..5ae4a35 100644
--- a/transaction/AccessMode.cpp
+++ b/transaction/AccessMode.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "AccessMode.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/AccessMode.hpp
----------------------------------------------------------------------
diff --git a/transaction/AccessMode.hpp b/transaction/AccessMode.hpp
index bb06689..d86cffb 100644
--- a/transaction/AccessMode.hpp
+++ b/transaction/AccessMode.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_ACCESS_MODE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/transaction/CMakeLists.txt b/transaction/CMakeLists.txt
index 430b4bc..263cb01 100644
--- a/transaction/CMakeLists.txt
+++ b/transaction/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 if (BUILD_SHARED_LIBS)
   set(GFLAGS_LIB_NAME gflags_nothreads-shared)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/CycleDetector.cpp
----------------------------------------------------------------------
diff --git a/transaction/CycleDetector.cpp b/transaction/CycleDetector.cpp
index b12897f..d7f3034 100644
--- a/transaction/CycleDetector.cpp
+++ b/transaction/CycleDetector.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/CycleDetector.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/CycleDetector.hpp
----------------------------------------------------------------------
diff --git a/transaction/CycleDetector.hpp b/transaction/CycleDetector.hpp
index 6865e2d..d0eba96 100644
--- a/transaction/CycleDetector.hpp
+++ b/transaction/CycleDetector.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_



[32/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSimpleTableReference.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSimpleTableReference.cpp b/parser/ParseSimpleTableReference.cpp
index e56af06..518dc20 100644
--- a/parser/ParseSimpleTableReference.cpp
+++ b/parser/ParseSimpleTableReference.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseSimpleTableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSimpleTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSimpleTableReference.hpp b/parser/ParseSimpleTableReference.hpp
index 5e6815e..4ff92a5 100644
--- a/parser/ParseSimpleTableReference.hpp
+++ b/parser/ParseSimpleTableReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_SIMPLE_TABLE_REFERENCE_HPP_
@@ -41,7 +43,7 @@ class ParseTreeNode;
 class ParseSimpleTableReference : public ParseTableReference {
  public:
   /**
-   * @brief Constructor. 
+   * @brief Constructor.
    * @note Takes ownership of \p table_name and \p sample.
    *
    * @param line_number The line number of the first token of the table reference.
@@ -97,4 +99,4 @@ class ParseSimpleTableReference : public ParseTableReference {
 
 }  // namespace quickstep
 
-#endif /* QUICKSTEP_PARSER_PARSE_SIMPLE_TABLE_REFERENCE_HPP_ */
+#endif  // QUICKSTEP_PARSER_PARSE_SIMPLE_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseStatement.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseStatement.hpp b/parser/ParseStatement.hpp
index 61475a9..cb5a1b5 100644
--- a/parser/ParseStatement.hpp
+++ b/parser/ParseStatement.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_STATEMENT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseString.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseString.cpp b/parser/ParseString.cpp
index 4770e1f..ca07541 100644
--- a/parser/ParseString.cpp
+++ b/parser/ParseString.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseString.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseString.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseString.hpp b/parser/ParseString.hpp
index af559c6..1997822 100644
--- a/parser/ParseString.hpp
+++ b/parser/ParseString.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_STRING_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSubqueryExpression.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryExpression.cpp b/parser/ParseSubqueryExpression.cpp
index 556b19d..991e00c 100644
--- a/parser/ParseSubqueryExpression.cpp
+++ b/parser/ParseSubqueryExpression.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseSubqueryExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSubqueryExpression.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryExpression.hpp b/parser/ParseSubqueryExpression.hpp
index d54d623..c7f9ccf 100644
--- a/parser/ParseSubqueryExpression.hpp
+++ b/parser/ParseSubqueryExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_SUBQUERY_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSubqueryTableReference.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryTableReference.cpp b/parser/ParseSubqueryTableReference.cpp
index 03a5f01..fb2a05d 100644
--- a/parser/ParseSubqueryTableReference.cpp
+++ b/parser/ParseSubqueryTableReference.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseSubqueryTableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSubqueryTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryTableReference.hpp b/parser/ParseSubqueryTableReference.hpp
index a8ca8ea..8168fc2 100644
--- a/parser/ParseSubqueryTableReference.hpp
+++ b/parser/ParseSubqueryTableReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_SUBQUERY_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseTableReference.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseTableReference.cpp b/parser/ParseTableReference.cpp
index 49384db..3ac02f7 100644
--- a/parser/ParseTableReference.cpp
+++ b/parser/ParseTableReference.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseTableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseTableReference.hpp b/parser/ParseTableReference.hpp
index c3500b0..9ef71e4 100644
--- a/parser/ParseTableReference.hpp
+++ b/parser/ParseTableReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseTreeNode.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseTreeNode.hpp b/parser/ParseTreeNode.hpp
index 51eed55..c9158f7 100644
--- a/parser/ParseTreeNode.hpp
+++ b/parser/ParseTreeNode.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_TREE_NODE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseWindow.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseWindow.hpp b/parser/ParseWindow.hpp
index 7b41683..5d7bf98 100644
--- a/parser/ParseWindow.hpp
+++ b/parser/ParseWindow.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_WINDOW_HPP_
@@ -42,7 +44,7 @@ struct ParseFrameInfo : ParseTreeNode {
    *                the current row in the frame.
    * @param num_follow The number of rows/value of range that is following
    *                   the current row in the frame.
-   **/  
+   **/
   ParseFrameInfo(const int line_number,
                  const int column_number,
                  const bool is_row_in,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParserModule.hpp
----------------------------------------------------------------------
diff --git a/parser/ParserModule.hpp b/parser/ParserModule.hpp
index c753095..4d45ed5 100644
--- a/parser/ParserModule.hpp
+++ b/parser/ParserModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Parser

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParserUtil.cpp
----------------------------------------------------------------------
diff --git a/parser/ParserUtil.cpp b/parser/ParserUtil.cpp
index 9c14eeb..743d10a 100644
--- a/parser/ParserUtil.cpp
+++ b/parser/ParserUtil.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParserUtil.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParserUtil.hpp
----------------------------------------------------------------------
diff --git a/parser/ParserUtil.hpp b/parser/ParserUtil.hpp
index b2fc798..fd993fe 100644
--- a/parser/ParserUtil.hpp
+++ b/parser/ParserUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSER_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index ec18f38..0953dc6 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 %option prefix="quickstep_yy"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 382ea44..3f0cc80 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 %name-prefix "quickstep_yy"
@@ -1178,7 +1178,7 @@ opt_priority_clause:
       }
     }
   };
-  
+
 with_clause:
   TOKEN_WITH with_list {
     $$ = $2;
@@ -1420,7 +1420,7 @@ window_declaration:
   TOKEN_WINDOW any_name TOKEN_AS '(' window_definition ')' {
     $$ = $5;
     $$->setName($2);
-  } 
+  }
 
 window_definition:
   opt_window_partition opt_window_order opt_window_frame {
@@ -1461,7 +1461,7 @@ frame_mode:
 
 frame_preceding:
   TOKEN_UNSIGNED_NUMVAL TOKEN_PRECEDING
-  | TOKEN_UNBOUNDED TOKEN_PRECEDING { 
+  | TOKEN_UNBOUNDED TOKEN_PRECEDING {
     $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "-1");
   }
   | TOKEN_CURRENT TOKEN_ROW {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/SqlParserWrapper.cpp
----------------------------------------------------------------------
diff --git a/parser/SqlParserWrapper.cpp b/parser/SqlParserWrapper.cpp
index bc9d87e..52a8910 100644
--- a/parser/SqlParserWrapper.cpp
+++ b/parser/SqlParserWrapper.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/SqlParserWrapper.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/SqlParserWrapper.hpp
----------------------------------------------------------------------
diff --git a/parser/SqlParserWrapper.hpp b/parser/SqlParserWrapper.hpp
index fbfb64e..e416e60 100644
--- a/parser/SqlParserWrapper.hpp
+++ b/parser/SqlParserWrapper.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_SQL_PARSER_WRAPPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index b6c747c..c17e1a1 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -980,22 +980,22 @@ static yyconst flex_int32_t yy_rule_can_match_eol[162] =
 #define YY_RESTORE_YY_MORE_OFFSET
 #line 1 "../SqlLexer.lpp"
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 #define YY_NO_INPUT 1
 #define YY_NO_UNISTD_H 1

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Aggregate.test
----------------------------------------------------------------------
diff --git a/parser/tests/Aggregate.test b/parser/tests/Aggregate.test
index 7962d3e..e7b8945 100644
--- a/parser/tests/Aggregate.test
+++ b/parser/tests/Aggregate.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 SELECT AGG(*), AGG(), AGG(a, b, c) FROM test
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Alter.test
----------------------------------------------------------------------
diff --git a/parser/tests/Alter.test b/parser/tests/Alter.test
index dd38eea..f050284 100644
--- a/parser/tests/Alter.test
+++ b/parser/tests/Alter.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 ALTER TABLE test
 ADD attr INT

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/tests/CMakeLists.txt b/parser/tests/CMakeLists.txt
index cea0f45..9b461ec 100644
--- a/parser/tests/CMakeLists.txt
+++ b/parser/tests/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_executable(quickstep_parser_tests_ParserTest
               "${CMAKE_CURRENT_SOURCE_DIR}/ParserTest.cpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Copy.test
----------------------------------------------------------------------
diff --git a/parser/tests/Copy.test b/parser/tests/Copy.test
index 15db147..dccaa82 100644
--- a/parser/tests/Copy.test
+++ b/parser/tests/Copy.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 COPY test FROM 'test.txt' WITH ()
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Create.test
----------------------------------------------------------------------
diff --git a/parser/tests/Create.test b/parser/tests/Create.test
index 2670c66..49e7ccd 100644
--- a/parser/tests/Create.test
+++ b/parser/tests/Create.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 CREATE TABLE TABLE (attr int)
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Delete.test
----------------------------------------------------------------------
diff --git a/parser/tests/Delete.test b/parser/tests/Delete.test
index e4bc335..da1c3f0 100644
--- a/parser/tests/Delete.test
+++ b/parser/tests/Delete.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 DELETE FROM test WHERE tb1=1
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Drop.test
----------------------------------------------------------------------
diff --git a/parser/tests/Drop.test b/parser/tests/Drop.test
index eec2a43..3735e5a 100644
--- a/parser/tests/Drop.test
+++ b/parser/tests/Drop.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 DROP TABLE test
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Index.test
----------------------------------------------------------------------
diff --git a/parser/tests/Index.test b/parser/tests/Index.test
index 39cd523..d97701b 100644
--- a/parser/tests/Index.test
+++ b/parser/tests/Index.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Name of index is required
 CREATE INDEX ON test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Insert.test
----------------------------------------------------------------------
diff --git a/parser/tests/Insert.test b/parser/tests/Insert.test
index a88e666..ef732b9 100644
--- a/parser/tests/Insert.test
+++ b/parser/tests/Insert.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 INSERT INTO test (attr1, attr2) VALUES (1, 2)
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Join.test
----------------------------------------------------------------------
diff --git a/parser/tests/Join.test b/parser/tests/Join.test
index 8a252a4..63edeb4 100644
--- a/parser/tests/Join.test
+++ b/parser/tests/Join.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 SELECT *
 FROM a JOIN b ON a.w = b.w

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/ParserTest.cpp
----------------------------------------------------------------------
diff --git a/parser/tests/ParserTest.cpp b/parser/tests/ParserTest.cpp
index ee3cb1a..cd2eaf4 100644
--- a/parser/tests/ParserTest.cpp
+++ b/parser/tests/ParserTest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <fstream>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Select.test
----------------------------------------------------------------------
diff --git a/parser/tests/Select.test b/parser/tests/Select.test
index 464c48f..b614a99 100644
--- a/parser/tests/Select.test
+++ b/parser/tests/Select.test
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 SELECT * FROM test
 --
@@ -1628,7 +1628,7 @@ SelectStatement
 
 # IN predicate
 SELECT *
-FROM test 
+FROM test
 WHERE col1 IN (1, 3, 5);
 --
 SelectStatement
@@ -1781,7 +1781,7 @@ SelectStatement
           +-AttributeReference[attribute_name=attr4]
 ==
 
-SELECT avg(attr1) OVER w FROM test 
+SELECT avg(attr1) OVER w FROM test
 WINDOW w AS
 (PARTITION BY attr2, attr3
  ORDER BY attr4 DESC NULLS FIRST, attr5 ASC NULLS LAST
@@ -1808,7 +1808,7 @@ SelectStatement
           +-AttributeReference[attribute_name=attr5]
 ==
 
-SELECT avg(attr1) OVER w1 AS avg1, sum(attr5) OVER w2 AS sum5 FROM test 
+SELECT avg(attr1) OVER w1 AS avg1, sum(attr5) OVER w2 AS sum5 FROM test
 WINDOW w1 AS
 (PARTITION BY attr2, attr3
  ORDER BY attr4
@@ -1847,7 +1847,7 @@ SelectStatement
           +-AttributeReference[attribute_name=attr6]
 ==
 
-SELECT avg(attr1) OVER 
+SELECT avg(attr1) OVER
 (PARTITION BY attr2, attr3
  ORDER BY attr4 DESC NULLS FIRST, attr5 ASC NULLS LAST
  RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/TPCH.test
----------------------------------------------------------------------
diff --git a/parser/tests/TPCH.test b/parser/tests/TPCH.test
index 7f4d911..e30d373 100644
--- a/parser/tests/TPCH.test
+++ b/parser/tests/TPCH.test
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Query 1
 SELECT

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/tests/Update.test
----------------------------------------------------------------------
diff --git a/parser/tests/Update.test b/parser/tests/Update.test
index 6ed31e6..4ae0fcb 100644
--- a/parser/tests/Update.test
+++ b/parser/tests/Update.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 UPDATE TABLE test SET attr = 1
 --

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/AdmitRequestMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/AdmitRequestMessage.hpp b/query_execution/AdmitRequestMessage.hpp
index e33b354..dd3d49b 100644
--- a/query_execution/AdmitRequestMessage.hpp
+++ b/query_execution/AdmitRequestMessage.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/BlockLocator.cpp
----------------------------------------------------------------------
diff --git a/query_execution/BlockLocator.cpp b/query_execution/BlockLocator.cpp
index 6cf5249..81684ba 100644
--- a/query_execution/BlockLocator.cpp
+++ b/query_execution/BlockLocator.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/BlockLocator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/BlockLocator.hpp
----------------------------------------------------------------------
diff --git a/query_execution/BlockLocator.hpp b/query_execution/BlockLocator.hpp
index bbd9b8f..a83a394 100644
--- a/query_execution/BlockLocator.hpp
+++ b/query_execution/BlockLocator.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_BLOCK_LOCATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index edbe5d0..4b180e3 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 QS_PROTOBUF_GENERATE_CPP(queryexecution_QueryContext_proto_srcs queryexecution_QueryContext_proto_hdrs
                          QueryContext.proto)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/ForemanBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanBase.hpp b/query_execution/ForemanBase.hpp
index 274b8fc..951f34d 100644
--- a/query_execution/ForemanBase.hpp
+++ b/query_execution/ForemanBase.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_FOREMAN_BASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index f935a0b..23db379 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/ForemanSingleNode.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/ForemanSingleNode.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.hpp b/query_execution/ForemanSingleNode.hpp
index d999095..71ce99d 100644
--- a/query_execution/ForemanSingleNode.hpp
+++ b/query_execution/ForemanSingleNode.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_FOREMAN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index 3371d6d..bf6edf9 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_execution/PolicyEnforcerBase.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/PolicyEnforcerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.hpp b/query_execution/PolicyEnforcerBase.hpp
index 15bc118..c75a531 100644
--- a/query_execution/PolicyEnforcerBase.hpp
+++ b/query_execution/PolicyEnforcerBase.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_BASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_execution/PolicyEnforcerSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerSingleNode.cpp b/query_execution/PolicyEnforcerSingleNode.cpp
index 549e39f..0bdb9b1 100644
--- a/query_execution/PolicyEnforcerSingleNode.cpp
+++ b/query_execution/PolicyEnforcerSingleNode.cpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 "query_execution/PolicyEnforcerSingleNode.hpp"



[45/50] [abbrv] incubator-quickstep git commit: Removed an unused message type.

Posted by zu...@apache.org.
Removed an unused message type.


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

Branch: refs/heads/travis-grpc
Commit: 6168996216af8278d5c789c67aa4ec8325fab483
Parents: 2c0ce6a
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 15:32:34 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 10:44:50 2016 -0700

----------------------------------------------------------------------
 query_execution/ForemanSingleNode.cpp        |  4 +---
 query_execution/PolicyEnforcerBase.cpp       | 13 -------------
 query_execution/QueryExecutionMessages.proto |  5 -----
 query_execution/QueryExecutionTypedefs.hpp   |  1 -
 query_execution/Shiftboss.cpp                |  1 -
 query_execution/Shiftboss.hpp                |  1 -
 query_execution/Worker.hpp                   |  1 -
 7 files changed, 1 insertion(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index 23db379..d064a6f 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -87,7 +87,6 @@ ForemanSingleNode::ForemanSingleNode(
       kPoisonMessage,
       kRebuildWorkOrderCompleteMessage,
       kWorkOrderFeedbackMessage,
-      kWorkOrdersAvailableMessage,
       kWorkOrderCompleteMessage};
 
   for (const auto message_type : receiver_message_types) {
@@ -122,8 +121,7 @@ void ForemanSingleNode::run() {
       case kDataPipelineMessage:
       case kRebuildWorkOrderCompleteMessage:
       case kWorkOrderCompleteMessage:
-      case kWorkOrderFeedbackMessage:
-      case kWorkOrdersAvailableMessage: {
+      case kWorkOrderFeedbackMessage: {
         policy_enforcer_->processMessage(tagged_message);
         break;
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index 78f7b44..4174bd6 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -107,19 +107,6 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
           op_index, proto.block_id(), proto.relation_id());
       break;
     }
-    case kWorkOrdersAvailableMessage: {
-      serialization::WorkOrdersAvailableMessage proto;
-      CHECK(proto.ParseFromArray(tagged_message.message(),
-                                 tagged_message.message_bytes()));
-      query_id = proto.query_id();
-      DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
-
-      op_index = proto.operator_index();
-
-      // Check if new work orders are available.
-      admitted_queries_[query_id]->fetchNormalWorkOrders(op_index);
-      break;
-    }
     case kWorkOrderFeedbackMessage: {
       WorkOrder::FeedbackMessage msg(
           const_cast<void *>(tagged_message.message()),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 20b684e..060efa1 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -74,11 +74,6 @@ message DataPipelineMessage {
   required uint64 query_id = 4;
 }
 
-message WorkOrdersAvailableMessage {
-  required uint64 operator_index = 1;
-  required uint64 query_id = 2;
-}
-
 // Distributed version related messages.
 message ShiftbossRegistrationMessage {
   // The total Work Order processing capacity in Shiftboss, which equals to the

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index d154d84..33a93b0 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -69,7 +69,6 @@ enum QueryExecutionMessageType : message_type_id {
   kWorkOrderCompleteMessage,  // From Worker to Foreman.
   kCatalogRelationNewBlockMessage,  // From InsertDestination to Foreman.
   kDataPipelineMessage,  // From InsertDestination or some WorkOrders to Foreman.
-  kWorkOrdersAvailableMessage,  // From some WorkOrders to Foreman.
   kWorkOrderFeedbackMessage,  // From some WorkOrders to Foreman on behalf of
                               // their corresponding RelationalOperators.
   kRebuildWorkOrderMessage,  // From Foreman to Worker.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 24c91fe..bd83dd4 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -152,7 +152,6 @@ void Shiftboss::run() {
       case kWorkOrderCompleteMessage:  // Fall through.
       case kRebuildWorkOrderCompleteMessage:
       case kDataPipelineMessage:
-      case kWorkOrdersAvailableMessage:
       case kWorkOrderFeedbackMessage: {
         LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
                   << "') forwarded typed '" << annotated_message.tagged_message.message_type()

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 9464a4d..30a8d1a 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -103,7 +103,6 @@ class Shiftboss : public Thread {
     // Message sent to Foreman.
     bus_->RegisterClientAsSender(shiftboss_client_id_, kCatalogRelationNewBlockMessage);
     bus_->RegisterClientAsSender(shiftboss_client_id_, kDataPipelineMessage);
-    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrdersAvailableMessage);
     bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderFeedbackMessage);
 
     // Forward the following message types from Foreman to Workers.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index 44a7447..aa39bb3 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -68,7 +68,6 @@ class Worker : public Thread {
                                  kRebuildWorkOrderCompleteMessage);
     bus_->RegisterClientAsSender(worker_client_id_, kCatalogRelationNewBlockMessage);
     bus_->RegisterClientAsSender(worker_client_id_, kDataPipelineMessage);
-    bus_->RegisterClientAsSender(worker_client_id_, kWorkOrdersAvailableMessage);
     bus_->RegisterClientAsSender(worker_client_id_, kWorkOrderFeedbackMessage);
 
     bus_->RegisterClientAsReceiver(worker_client_id_, kWorkOrderMessage);


[12/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/TypedValue_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/TypedValue_unittest.cpp b/types/tests/TypedValue_unittest.cpp
index f2cbb01..0734c26 100644
--- a/types/tests/TypedValue_unittest.cpp
+++ b/types/tests/TypedValue_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 // TODO(chasseur): Trim down duplicated code in this file using more

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/VarCharType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/VarCharType_unittest.cpp b/types/tests/VarCharType_unittest.cpp
index aaca906..57ed10a 100644
--- a/types/tests/VarCharType_unittest.cpp
+++ b/types/tests/VarCharType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/YearMonthIntervalType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/YearMonthIntervalType_unittest.cpp b/types/tests/YearMonthIntervalType_unittest.cpp
index 40d08de..2fdb266 100644
--- a/types/tests/YearMonthIntervalType_unittest.cpp
+++ b/types/tests/YearMonthIntervalType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/Alignment.hpp
----------------------------------------------------------------------
diff --git a/utility/Alignment.hpp b/utility/Alignment.hpp
index 1a3ff4b..62e015a 100644
--- a/utility/Alignment.hpp
+++ b/utility/Alignment.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_ALIGNMENT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/BitManipulation.hpp
----------------------------------------------------------------------
diff --git a/utility/BitManipulation.hpp b/utility/BitManipulation.hpp
index e06fdb7..3f7d841 100644
--- a/utility/BitManipulation.hpp
+++ b/utility/BitManipulation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_BIT_MANIPULATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/BitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BitVector.hpp b/utility/BitVector.hpp
index a6197a5..bb76315 100644
--- a/utility/BitVector.hpp
+++ b/utility/BitVector.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_BIT_VECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/BloomFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.hpp b/utility/BloomFilter.hpp
index b93df84..8d62da9 100644
--- a/utility/BloomFilter.hpp
+++ b/utility/BloomFilter.hpp
@@ -1,24 +1,21 @@
 /**
-*   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-*     University of Wisconsin\u2014Madison.
-*
-*   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.
-*
-*   The hashing function used by the simple Bloom Filter implementation below
-*   is a slightly modified version of the original hashing function, which was
-*   written by Arash Partow and placed in public domain. The Quickstep team
-*   hereby disclaims the credits to this part of the code.
-**/
+ * 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 QUICKSTEP_UTILITY_BLOOM_FILTER_HPP
 #define QUICKSTEP_UTILITY_BLOOM_FILTER_HPP

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/BloomFilter.proto
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.proto b/utility/BloomFilter.proto
index 8dd9163..0f67878 100644
--- a/utility/BloomFilter.proto
+++ b/utility/BloomFilter.proto
@@ -1,17 +1,19 @@
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//     University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 803b909..ae1179d 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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(CheckCXXSourceCompiles)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/CalculateInstalledMemory.cpp
----------------------------------------------------------------------
diff --git a/utility/CalculateInstalledMemory.cpp b/utility/CalculateInstalledMemory.cpp
index ee90db2..f2a8982 100644
--- a/utility/CalculateInstalledMemory.cpp
+++ b/utility/CalculateInstalledMemory.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/CalculateInstalledMemory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/CalculateInstalledMemory.hpp
----------------------------------------------------------------------
diff --git a/utility/CalculateInstalledMemory.hpp b/utility/CalculateInstalledMemory.hpp
index 9bbfc71..44081ed 100644
--- a/utility/CalculateInstalledMemory.hpp
+++ b/utility/CalculateInstalledMemory.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_SYSTEM_CALCULATE_INSTALLED_MEMORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/Cast.hpp
----------------------------------------------------------------------
diff --git a/utility/Cast.hpp b/utility/Cast.hpp
index 1a94e48..7d2a8c0 100644
--- a/utility/Cast.hpp
+++ b/utility/Cast.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_CAST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/CheckSnprintf.hpp
----------------------------------------------------------------------
diff --git a/utility/CheckSnprintf.hpp b/utility/CheckSnprintf.hpp
index d66aa7d..21abc94 100644
--- a/utility/CheckSnprintf.hpp
+++ b/utility/CheckSnprintf.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_CHECK_SNPRINTF_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/DAG.hpp
----------------------------------------------------------------------
diff --git a/utility/DAG.hpp b/utility/DAG.hpp
index 7955772..a1f2619 100644
--- a/utility/DAG.hpp
+++ b/utility/DAG.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_DAG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/EqualsAnyConstant.hpp
----------------------------------------------------------------------
diff --git a/utility/EqualsAnyConstant.hpp b/utility/EqualsAnyConstant.hpp
index cf6ac29..3c12bf1 100644
--- a/utility/EqualsAnyConstant.hpp
+++ b/utility/EqualsAnyConstant.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_EQUALS_ANY_CONSTANT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/ExecutionDAGVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/ExecutionDAGVisualizer.cpp b/utility/ExecutionDAGVisualizer.cpp
index 0c0bbb1..37d00cd 100644
--- a/utility/ExecutionDAGVisualizer.cpp
+++ b/utility/ExecutionDAGVisualizer.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/ExecutionDAGVisualizer.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/ExecutionDAGVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/ExecutionDAGVisualizer.hpp b/utility/ExecutionDAGVisualizer.hpp
index 5c9e434..b869bce 100644
--- a/utility/ExecutionDAGVisualizer.hpp
+++ b/utility/ExecutionDAGVisualizer.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_EXECUTION_DAG_VISUALIZER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/Glob.cpp
----------------------------------------------------------------------
diff --git a/utility/Glob.cpp b/utility/Glob.cpp
index 7e49c13..9287ed9 100644
--- a/utility/Glob.cpp
+++ b/utility/Glob.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/Glob.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/Glob.hpp
----------------------------------------------------------------------
diff --git a/utility/Glob.hpp b/utility/Glob.hpp
index 842e760..ef14c22 100644
--- a/utility/Glob.hpp
+++ b/utility/Glob.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_GLOB_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/HashPair.hpp
----------------------------------------------------------------------
diff --git a/utility/HashPair.hpp b/utility/HashPair.hpp
index 04e9457..8ad7f15 100644
--- a/utility/HashPair.hpp
+++ b/utility/HashPair.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_HASH_PAIR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/Macros.hpp
----------------------------------------------------------------------
diff --git a/utility/Macros.hpp b/utility/Macros.hpp
index 3edbebd..9eb1435 100644
--- a/utility/Macros.hpp
+++ b/utility/Macros.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_MACROS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/MemStream.hpp
----------------------------------------------------------------------
diff --git a/utility/MemStream.hpp b/utility/MemStream.hpp
index f15e1a0..30678c0 100644
--- a/utility/MemStream.hpp
+++ b/utility/MemStream.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_MEMSTREAM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 962d577..50cf7f0 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/PlanVisualizer.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PlanVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.hpp b/utility/PlanVisualizer.hpp
index 080b7de..1c0df77 100644
--- a/utility/PlanVisualizer.hpp
+++ b/utility/PlanVisualizer.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_PLAN_VISUALIZER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PrimeNumber.cpp
----------------------------------------------------------------------
diff --git a/utility/PrimeNumber.cpp b/utility/PrimeNumber.cpp
index d1cc87d..b4764d5 100644
--- a/utility/PrimeNumber.cpp
+++ b/utility/PrimeNumber.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/PrimeNumber.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PrimeNumber.hpp
----------------------------------------------------------------------
diff --git a/utility/PrimeNumber.hpp b/utility/PrimeNumber.hpp
index 8aa39e8..5c86772 100644
--- a/utility/PrimeNumber.hpp
+++ b/utility/PrimeNumber.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_PRIME_NUMBER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PtrList.hpp
----------------------------------------------------------------------
diff --git a/utility/PtrList.hpp b/utility/PtrList.hpp
index a3ac3c2..5e34ec8 100644
--- a/utility/PtrList.hpp
+++ b/utility/PtrList.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_PTR_LIST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PtrMap.hpp
----------------------------------------------------------------------
diff --git a/utility/PtrMap.hpp b/utility/PtrMap.hpp
index 7dd11b9..f079099 100644
--- a/utility/PtrMap.hpp
+++ b/utility/PtrMap.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_PTR_MAP_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/PtrVector.hpp
----------------------------------------------------------------------
diff --git a/utility/PtrVector.hpp b/utility/PtrVector.hpp
index af20e61..0ca8a0b 100644
--- a/utility/PtrVector.hpp
+++ b/utility/PtrVector.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_PTR_VECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/ScopedBuffer.hpp
----------------------------------------------------------------------
diff --git a/utility/ScopedBuffer.hpp b/utility/ScopedBuffer.hpp
index 03290ff..df35f78 100644
--- a/utility/ScopedBuffer.hpp
+++ b/utility/ScopedBuffer.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_SCOPED_BUFFER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/ScopedDeleter.hpp
----------------------------------------------------------------------
diff --git a/utility/ScopedDeleter.hpp b/utility/ScopedDeleter.hpp
index 42830b8..050b536 100644
--- a/utility/ScopedDeleter.hpp
+++ b/utility/ScopedDeleter.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_SCOPED_DELETER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/ShardedLockManager.hpp
----------------------------------------------------------------------
diff --git a/utility/ShardedLockManager.hpp b/utility/ShardedLockManager.hpp
index e3eba85..520f879 100644
--- a/utility/ShardedLockManager.hpp
+++ b/utility/ShardedLockManager.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_SHARDED_LOCK_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/SortConfiguration.cpp
----------------------------------------------------------------------
diff --git a/utility/SortConfiguration.cpp b/utility/SortConfiguration.cpp
index 238b569..01bb01b 100644
--- a/utility/SortConfiguration.cpp
+++ b/utility/SortConfiguration.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/SortConfiguration.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/SortConfiguration.hpp
----------------------------------------------------------------------
diff --git a/utility/SortConfiguration.hpp b/utility/SortConfiguration.hpp
index 473444e..d12ec8f 100644
--- a/utility/SortConfiguration.hpp
+++ b/utility/SortConfiguration.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_SORTCONFIGURATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/SortConfiguration.proto
----------------------------------------------------------------------
diff --git a/utility/SortConfiguration.proto b/utility/SortConfiguration.proto
index ce7738e..23c1e42 100644
--- a/utility/SortConfiguration.proto
+++ b/utility/SortConfiguration.proto
@@ -1,16 +1,19 @@
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/SqlError.cpp
----------------------------------------------------------------------
diff --git a/utility/SqlError.cpp b/utility/SqlError.cpp
index 775b5d2..8c7c650 100644
--- a/utility/SqlError.cpp
+++ b/utility/SqlError.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/SqlError.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/SqlError.hpp
----------------------------------------------------------------------
diff --git a/utility/SqlError.hpp b/utility/SqlError.hpp
index 2396e19..e10b5b7 100644
--- a/utility/SqlError.hpp
+++ b/utility/SqlError.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_SQL_ERROR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/StringUtil.cpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.cpp b/utility/StringUtil.cpp
index b38e8f4..49ef337 100644
--- a/utility/StringUtil.cpp
+++ b/utility/StringUtil.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #define __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/StringUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.hpp b/utility/StringUtil.hpp
index 6477ded..9aa57ab 100644
--- a/utility/StringUtil.hpp
+++ b/utility/StringUtil.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_STRING_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/TemplateUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/TemplateUtil.hpp b/utility/TemplateUtil.hpp
index 60022f5..33e4f42 100644
--- a/utility/TemplateUtil.hpp
+++ b/utility/TemplateUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_TEMPLATE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/ThreadSafeQueue.hpp
----------------------------------------------------------------------
diff --git a/utility/ThreadSafeQueue.hpp b/utility/ThreadSafeQueue.hpp
index 4581dc3..440f8a7 100644
--- a/utility/ThreadSafeQueue.hpp
+++ b/utility/ThreadSafeQueue.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_THREAD_SAFE_QUEUE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/TreeStringSerializable.hpp
----------------------------------------------------------------------
diff --git a/utility/TreeStringSerializable.hpp b/utility/TreeStringSerializable.hpp
index 96f4fcf..8564fb9 100644
--- a/utility/TreeStringSerializable.hpp
+++ b/utility/TreeStringSerializable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_TREE_STRING_SERIALIZABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/UtilityConfig.h.in
----------------------------------------------------------------------
diff --git a/utility/UtilityConfig.h.in b/utility/UtilityConfig.h.in
index 1661fa1..3e291b4 100644
--- a/utility/UtilityConfig.h.in
+++ b/utility/UtilityConfig.h.in
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_HAVE_ALIGNED_ALLOC

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/UtilityModule.hpp
----------------------------------------------------------------------
diff --git a/utility/UtilityModule.hpp b/utility/UtilityModule.hpp
index 3d28f3a..e3c060b 100644
--- a/utility/UtilityModule.hpp
+++ b/utility/UtilityModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Utility

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/VectorUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/VectorUtil.hpp b/utility/VectorUtil.hpp
index 8559a46..0cea2fc 100644
--- a/utility/VectorUtil.hpp
+++ b/utility/VectorUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_VECTOR_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/BitVector_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/BitVector_unittest.cpp b/utility/tests/BitVector_unittest.cpp
index 52b475a..774b830 100644
--- a/utility/tests/BitVector_unittest.cpp
+++ b/utility/tests/BitVector_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/BloomFilter_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/BloomFilter_unittest.cpp b/utility/tests/BloomFilter_unittest.cpp
index 4d8beae..fa4b873 100644
--- a/utility/tests/BloomFilter_unittest.cpp
+++ b/utility/tests/BloomFilter_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstdint>



[36/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/DropRelation.hpp
----------------------------------------------------------------------
diff --git a/cli/DropRelation.hpp b/cli/DropRelation.hpp
index e82e246..91cea43 100644
--- a/cli/DropRelation.hpp
+++ b/cli/DropRelation.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_DROP_RELATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/InputParserUtil.cpp
----------------------------------------------------------------------
diff --git a/cli/InputParserUtil.cpp b/cli/InputParserUtil.cpp
index 352883e..0538afc 100644
--- a/cli/InputParserUtil.cpp
+++ b/cli/InputParserUtil.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/InputParserUtil.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/InputParserUtil.hpp
----------------------------------------------------------------------
diff --git a/cli/InputParserUtil.hpp b/cli/InputParserUtil.hpp
index ebb32d2..d34dbed 100644
--- a/cli/InputParserUtil.hpp
+++ b/cli/InputParserUtil.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_INPUT_PARSER_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/LineReader.cpp
----------------------------------------------------------------------
diff --git a/cli/LineReader.cpp b/cli/LineReader.cpp
index d7ff3c2..002727d 100644
--- a/cli/LineReader.cpp
+++ b/cli/LineReader.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/LineReader.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/LineReader.hpp
----------------------------------------------------------------------
diff --git a/cli/LineReader.hpp b/cli/LineReader.hpp
index 2aecac5..867d676 100644
--- a/cli/LineReader.hpp
+++ b/cli/LineReader.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_LINE_READER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/LineReaderDumb.cpp
----------------------------------------------------------------------
diff --git a/cli/LineReaderDumb.cpp b/cli/LineReaderDumb.cpp
index 287bb3b..36edf73 100644
--- a/cli/LineReaderDumb.cpp
+++ b/cli/LineReaderDumb.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/LineReaderDumb.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/LineReaderDumb.hpp
----------------------------------------------------------------------
diff --git a/cli/LineReaderDumb.hpp b/cli/LineReaderDumb.hpp
index e1158e9..080268a 100644
--- a/cli/LineReaderDumb.hpp
+++ b/cli/LineReaderDumb.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_LINE_READER_DUMB_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/LineReaderLineNoise.cpp
----------------------------------------------------------------------
diff --git a/cli/LineReaderLineNoise.cpp b/cli/LineReaderLineNoise.cpp
index 7856e9c..9c3650e 100644
--- a/cli/LineReaderLineNoise.cpp
+++ b/cli/LineReaderLineNoise.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/LineReaderLineNoise.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/LineReaderLineNoise.hpp
----------------------------------------------------------------------
diff --git a/cli/LineReaderLineNoise.hpp b/cli/LineReaderLineNoise.hpp
index d22fa30..ec256f6 100644
--- a/cli/LineReaderLineNoise.hpp
+++ b/cli/LineReaderLineNoise.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_LINE_READER_LINE_NOISE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/PrintToScreen.cpp
----------------------------------------------------------------------
diff --git a/cli/PrintToScreen.cpp b/cli/PrintToScreen.cpp
index 76e90eb..7d06474 100644
--- a/cli/PrintToScreen.cpp
+++ b/cli/PrintToScreen.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/PrintToScreen.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/PrintToScreen.hpp
----------------------------------------------------------------------
diff --git a/cli/PrintToScreen.hpp b/cli/PrintToScreen.hpp
index 6a29426..2b5fd7e 100644
--- a/cli/PrintToScreen.hpp
+++ b/cli/PrintToScreen.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_PRINT_TO_SCREEN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 78d7765..8269197 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /* A standalone command-line interface to QuickStep */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/tests/CMakeLists.txt b/cli/tests/CMakeLists.txt
index 7da56d1..3bceba8 100644
--- a/cli/tests/CMakeLists.txt
+++ b/cli/tests/CMakeLists.txt
@@ -1,18 +1,20 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
- 
+# 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.
+
 add_subdirectory(command_executor)
 
 add_executable(quickstep_cli_tests_CommandExecutorTest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/CommandExecutorTest.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTest.cpp b/cli/tests/CommandExecutorTest.cpp
index 7b78cff..6ad2183 100644
--- a/cli/tests/CommandExecutorTest.cpp
+++ b/cli/tests/CommandExecutorTest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/CommandExecutorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.cpp b/cli/tests/CommandExecutorTestRunner.cpp
index 56f4ab9..9c701cd 100644
--- a/cli/tests/CommandExecutorTestRunner.cpp
+++ b/cli/tests/CommandExecutorTestRunner.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/tests/CommandExecutorTestRunner.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/CommandExecutorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.hpp b/cli/tests/CommandExecutorTestRunner.hpp
index 69692ae..682dc81 100644
--- a/cli/tests/CommandExecutorTestRunner.hpp
+++ b/cli/tests/CommandExecutorTestRunner.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_TESTS_COMMAND_EXECUTOR_TEST_RUNNER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/command_executor/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/tests/command_executor/CMakeLists.txt b/cli/tests/command_executor/CMakeLists.txt
index c9bcb36..0bdf865 100644
--- a/cli/tests/command_executor/CMakeLists.txt
+++ b/cli/tests/command_executor/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_test(quickstep_cli_tests_commandexecutor_d
          "../quickstep_cli_tests_CommandExecutorTest"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/command_executor/D.test
----------------------------------------------------------------------
diff --git a/cli/tests/command_executor/D.test b/cli/tests/command_executor/D.test
index d18ab47..1b35b58 100644
--- a/cli/tests/command_executor/D.test
+++ b/cli/tests/command_executor/D.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 CREATE TABLE foo (col1 INT,
                   col2 LONG,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/tests/command_executor/Dt.test
----------------------------------------------------------------------
diff --git a/cli/tests/command_executor/Dt.test b/cli/tests/command_executor/Dt.test
index 65c6be3..8d81029 100644
--- a/cli/tests/command_executor/Dt.test
+++ b/cli/tests/command_executor/Dt.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 CREATE TABLE foo (col1 INT,
                   col2 LONG,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cmake/FindGSasl.cmake
----------------------------------------------------------------------
diff --git a/cmake/FindGSasl.cmake b/cmake/FindGSasl.cmake
index f6bab19..28fb5b8 100644
--- a/cmake/FindGSasl.cmake
+++ b/cmake/FindGSasl.cmake
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # - Try to find the GNU sasl library (gsasl)
 #

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cmake/FindKerberos.cmake
----------------------------------------------------------------------
diff --git a/cmake/FindKerberos.cmake b/cmake/FindKerberos.cmake
index c944880..11835fb 100644
--- a/cmake/FindKerberos.cmake
+++ b/cmake/FindKerberos.cmake
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # - Find kerberos
 # Find the native KERBEROS includes and library

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cmake/FindLibNuma.cmake
----------------------------------------------------------------------
diff --git a/cmake/FindLibNuma.cmake b/cmake/FindLibNuma.cmake
index 5280a6c..47a65f4 100644
--- a/cmake/FindLibNuma.cmake
+++ b/cmake/FindLibNuma.cmake
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find LibNuma
 find_path(LIBNUMA_INCLUDE_DIR NAMES numa.h numaif.h)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cmake/FindLibhdfs3.cmake
----------------------------------------------------------------------
diff --git a/cmake/FindLibhdfs3.cmake b/cmake/FindLibhdfs3.cmake
index 17f4fd4..7bd7c7c 100644
--- a/cmake/FindLibhdfs3.cmake
+++ b/cmake/FindLibhdfs3.cmake
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find the Pivotal libhdfs3.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/compression/CMakeLists.txt b/compression/CMakeLists.txt
index cb5c5e7..ede899c 100644
--- a/compression/CMakeLists.txt
+++ b/compression/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_compression_CompressionDictionary

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionDictionary.cpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionary.cpp b/compression/CompressionDictionary.cpp
index 8043857..72c183f 100644
--- a/compression/CompressionDictionary.cpp
+++ b/compression/CompressionDictionary.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "compression/CompressionDictionary.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionDictionary.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionary.hpp b/compression/CompressionDictionary.hpp
index fe2cea0..27592e8 100644
--- a/compression/CompressionDictionary.hpp
+++ b/compression/CompressionDictionary.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_COMPRESSION_COMPRESSION_DICTIONARY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionDictionaryBuilder.cpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryBuilder.cpp b/compression/CompressionDictionaryBuilder.cpp
index 8358704..905af91 100644
--- a/compression/CompressionDictionaryBuilder.cpp
+++ b/compression/CompressionDictionaryBuilder.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "compression/CompressionDictionaryBuilder.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionDictionaryBuilder.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryBuilder.hpp b/compression/CompressionDictionaryBuilder.hpp
index ff10887..cad6852 100644
--- a/compression/CompressionDictionaryBuilder.hpp
+++ b/compression/CompressionDictionaryBuilder.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_COMPRESSION_COMPRESSION_DICTIONARY_BUILDER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionDictionaryLite.cpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryLite.cpp b/compression/CompressionDictionaryLite.cpp
index a2a54ef..d21d43c 100644
--- a/compression/CompressionDictionaryLite.cpp
+++ b/compression/CompressionDictionaryLite.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "compression/CompressionDictionaryLite.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionDictionaryLite.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryLite.hpp b/compression/CompressionDictionaryLite.hpp
index 45019c0..c4c338e 100644
--- a/compression/CompressionDictionaryLite.hpp
+++ b/compression/CompressionDictionaryLite.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_COMPRESSION_COMPRESSION_DICTIONARY_LITE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/CompressionModule.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionModule.hpp b/compression/CompressionModule.hpp
index 0aaefe6..f5e3e9b 100644
--- a/compression/CompressionModule.hpp
+++ b/compression/CompressionModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Compression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/compression/tests/CompressionDictionary_unittest.cpp
----------------------------------------------------------------------
diff --git a/compression/tests/CompressionDictionary_unittest.cpp b/compression/tests/CompressionDictionary_unittest.cpp
index cce78d4..92bd361 100644
--- a/compression/tests/CompressionDictionary_unittest.cpp
+++ b/compression/tests/CompressionDictionary_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cyclic_dependency.py
----------------------------------------------------------------------
diff --git a/cyclic_dependency.py b/cyclic_dependency.py
old mode 100755
new mode 100644
index a5cca25..adb5fc7
--- a/cyclic_dependency.py
+++ b/cyclic_dependency.py
@@ -18,20 +18,22 @@
 #       cyclic_dependency.py --cycles
 #
 
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 from __future__ import absolute_import
 from __future__ import division

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/empty_src.cpp
----------------------------------------------------------------------
diff --git a/empty_src.cpp b/empty_src.cpp
index 3a1c053..54054d0 100644
--- a/empty_src.cpp
+++ b/empty_src.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 // On OSX, ranlib complains if a static library archive contains no symbols,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/CMakeLists.txt b/expressions/CMakeLists.txt
index 6ef3c24..b1f1fb1 100644
--- a/expressions/CMakeLists.txt
+++ b/expressions/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_subdirectory(aggregation)
 add_subdirectory(predicate)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/ExpressionFactories.cpp
----------------------------------------------------------------------
diff --git a/expressions/ExpressionFactories.cpp b/expressions/ExpressionFactories.cpp
index f76e699..01d22a0 100644
--- a/expressions/ExpressionFactories.cpp
+++ b/expressions/ExpressionFactories.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/ExpressionFactories.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/ExpressionFactories.hpp
----------------------------------------------------------------------
diff --git a/expressions/ExpressionFactories.hpp b/expressions/ExpressionFactories.hpp
index 0c89513..f959dff 100644
--- a/expressions/ExpressionFactories.hpp
+++ b/expressions/ExpressionFactories.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_EXPRESSION_FACTORIES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/Expressions.proto
----------------------------------------------------------------------
diff --git a/expressions/Expressions.proto b/expressions/Expressions.proto
index ca688b5..8d923c5 100644
--- a/expressions/Expressions.proto
+++ b/expressions/Expressions.proto
@@ -1,17 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/ExpressionsModule.hpp
----------------------------------------------------------------------
diff --git a/expressions/ExpressionsModule.hpp b/expressions/ExpressionsModule.hpp
index 6d6fc5a..5302be7 100644
--- a/expressions/ExpressionsModule.hpp
+++ b/expressions/ExpressionsModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Expressions

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunction.cpp b/expressions/aggregation/AggregateFunction.cpp
index 055c6f7..63a297e 100644
--- a/expressions/aggregation/AggregateFunction.cpp
+++ b/expressions/aggregation/AggregateFunction.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunction.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunction.hpp b/expressions/aggregation/AggregateFunction.hpp
index bda156d..cccc699 100644
--- a/expressions/aggregation/AggregateFunction.hpp
+++ b/expressions/aggregation/AggregateFunction.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_HPP_
@@ -48,7 +51,7 @@ class Type;
  * AggregationHandle object, then use the methods of AggregationHandle to do
  * the actual aggregation, then finally delete the AggregationHandle when
  * finished.
- * 
+ *
  * See AggregationHandle for more detailed information about how aggregates are
  * actually computed.
  **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunction.proto
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunction.proto b/expressions/aggregation/AggregateFunction.proto
index 1fc09d4..07e0e30 100644
--- a/expressions/aggregation/AggregateFunction.proto
+++ b/expressions/aggregation/AggregateFunction.proto
@@ -1,16 +1,19 @@
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionAvg.cpp b/expressions/aggregation/AggregateFunctionAvg.cpp
index 76cfbb1..040d7d9 100644
--- a/expressions/aggregation/AggregateFunctionAvg.cpp
+++ b/expressions/aggregation/AggregateFunctionAvg.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunctionAvg.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionAvg.hpp b/expressions/aggregation/AggregateFunctionAvg.hpp
index b43d9c4..9d08d1c 100644
--- a/expressions/aggregation/AggregateFunctionAvg.hpp
+++ b/expressions/aggregation/AggregateFunctionAvg.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_AVG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionCount.cpp b/expressions/aggregation/AggregateFunctionCount.cpp
index de404a4..466ff2f 100644
--- a/expressions/aggregation/AggregateFunctionCount.cpp
+++ b/expressions/aggregation/AggregateFunctionCount.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunctionCount.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionCount.hpp b/expressions/aggregation/AggregateFunctionCount.hpp
index c45bf9e..ee34de4 100644
--- a/expressions/aggregation/AggregateFunctionCount.hpp
+++ b/expressions/aggregation/AggregateFunctionCount.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_COUNT_HPP_



[42/50] [abbrv] incubator-quickstep git commit: Added guards for gnu_source definition in CMake.

Posted by zu...@apache.org.
Added guards for gnu_source definition in CMake.


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

Branch: refs/heads/travis-grpc
Commit: 658cb61760ab6bc98ead1faf231b7b7f27a0fa1e
Parents: 3c2749e
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Aug 8 12:19:00 2016 -0500
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:42:51 2016 -0700

----------------------------------------------------------------------
 storage/CMakeLists.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/658cb617/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index c4b8f70..65a7975 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -50,7 +50,9 @@ endif()
 # See if mmap can be used to allocate Linux hugetlb pages.
 include(CheckCXXSourceCompiles)
 CHECK_CXX_SOURCE_COMPILES("
+  #ifndef _GNU_SOURCE
   #define _GNU_SOURCE
+  #endif
   #include <sys/mman.h>
 
   int main() {


[46/50] [abbrv] incubator-quickstep git commit: Logged all sent messages using glog.

Posted by zu...@apache.org.
Logged all sent messages using glog.


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

Branch: refs/heads/travis-grpc
Commit: d9135a8a2d11a1eabf6705c88391c498f4be38bb
Parents: 6168996
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 22:49:59 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 19:48:01 2016 -0700

----------------------------------------------------------------------
 query_execution/ForemanSingleNode.cpp         | 14 ++--
 query_execution/PolicyEnforcerDistributed.cpp | 38 ++++------
 query_execution/QueryExecutionUtil.hpp        |  6 +-
 query_execution/Shiftboss.cpp                 | 86 +++++++++-------------
 query_execution/Worker.cpp                    |  6 +-
 relational_operators/DeleteOperator.cpp       | 10 +--
 relational_operators/RebuildWorkOrder.hpp     |  7 +-
 relational_operators/UpdateOperator.cpp       | 10 +--
 relational_operators/WorkOrder.hpp            |  7 +-
 storage/InsertDestination.cpp                 | 18 +++--
 storage/InsertDestination.hpp                 | 10 +--
 11 files changed, 92 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index d064a6f..7596b00 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -168,16 +168,15 @@ void ForemanSingleNode::run() {
       // Signal the main thread that there are no queries to be executed.
       // Currently the message doesn't have any real content.
       TaggedMessage completion_tagged_message(kWorkloadCompletionMessage);
+      DLOG(INFO) << "ForemanSingleNode sent WorkloadCompletionMessage (typed '" << kWorkloadCompletionMessage
+                 << "') to CLI with TMB client ID " << main_thread_client_id_;
       const tmb::MessageBus::SendStatus send_status =
           QueryExecutionUtil::SendTMBMessage(
               bus_,
               foreman_client_id_,
               main_thread_client_id_,
               move(completion_tagged_message));
-      CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-          << "Message could not be sent from Foreman with TMB client ID "
-          << foreman_client_id_ << " to main thread with TMB client ID"
-          << main_thread_client_id_;
+      CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
     }
   }
 }
@@ -225,15 +224,14 @@ void ForemanSingleNode::sendWorkerMessage(const size_t worker_thread_index,
   }
   TaggedMessage worker_tagged_message(&message, sizeof(message), type);
 
+  DLOG(INFO) << "ForemanSingleNode sent WorkOrderMessage (typed '" << type
+             << "') to Worker with TMB client ID " << worker_directory_->getClientID(worker_thread_index);
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
                                          worker_directory_->getClientID(worker_thread_index),
                                          move(worker_tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-      "Message could not be sent from Foreman with TMB client ID "
-      << foreman_client_id_ << " to Foreman with TMB client ID "
-      << worker_directory_->getClientID(worker_thread_index);
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 const std::vector<WorkOrderTimeEntry>& ForemanSingleNode

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 6d0de47..c76a9e1 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -170,25 +170,22 @@ void PolicyEnforcerDistributed::initiateQueryInShiftboss(QueryHandle *query_hand
                         kQueryInitiateMessage);
   free(proto_bytes);
 
-  LOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
-            << "') to Shiftboss 0";
-
   // TODO(zuyu): Multiple Shiftbosses support.
+  DLOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
+             << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
                                          shiftboss_directory_->getClientId(0),
                                          move(message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   // Wait Shiftboss for QueryInitiateResponseMessage.
   const tmb::AnnotatedMessage annotated_message = bus_->Receive(foreman_client_id_, 0, true);
   const TaggedMessage &tagged_message = annotated_message.tagged_message;
   DCHECK_EQ(kQueryInitiateResponseMessage, tagged_message.message_type());
-  LOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
-            << "' message from client " << annotated_message.sender;
+  DLOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
+             << "' message from client " << annotated_message.sender;
 
   S::QueryInitiateResponseMessage proto_response;
   CHECK(proto_response.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
@@ -215,30 +212,27 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
                           kQueryTeardownMessage);
 
     // TODO(zuyu): Support multiple shiftbosses.
-    LOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
-              << "') to Shiftboss 0";
+    DLOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
+               << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
     tmb::MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            foreman_client_id_,
                                            shiftboss_directory_->getClientId(0),
                                            move(message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-        << " to Shiftboss";
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
     TaggedMessage cli_message(kQueryExecutionSuccessMessage);
 
     // Notify the CLI query execution successfully.
-    LOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '" << kQueryExecutionSuccessMessage
-              << "') to CLI with TMB client id " << cli_id;
+    DLOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '"
+               << kQueryExecutionSuccessMessage
+               << "') to CLI with TMB client id " << cli_id;
     send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            foreman_client_id_,
                                            cli_id,
                                            move(cli_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-        << " to CLI with TMB client ID " << cli_id;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
     return;
   }
 
@@ -263,17 +257,15 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
                         kSaveQueryResultMessage);
   free(proto_bytes);
 
-  LOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
-            << "') to Shiftboss 0";
   // TODO(zuyu): Support multiple shiftbosses.
+  DLOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
+             << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
                                          shiftboss_directory_->getClientId(0),
                                          move(message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index 906fb6b..feb4cc0 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -128,11 +128,11 @@ class QueryExecutionUtil {
     address.All(true);
     TaggedMessage poison_tagged_message(kPoisonMessage);
 
+    DLOG(INFO) << "TMB client ID " << sender_id
+               << " broadcast PoisonMessage (typed '" << kPoisonMessage << "') to all";
     const tmb::MessageBus::SendStatus send_status = bus->Send(
         sender_id, address, style, std::move(poison_tagged_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-       "Broadcast poison message from sender with TMB client ID " << sender_id
-       << " failed";
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index bd83dd4..ddfd47f 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -73,9 +73,9 @@ void Shiftboss::run() {
     // Receive() is a blocking call, causing this thread to sleep until next
     // message is received.
     AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
-    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-              << "') received the typed '" << annotated_message.tagged_message.message_type()
-              << "' message from client " << annotated_message.sender;
+    DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+               << "') received the typed '" << annotated_message.tagged_message.message_type()
+               << "' message from client " << annotated_message.sender;
     switch (annotated_message.tagged_message.message_type()) {
       case kShiftbossRegistrationResponseMessage: {
         foreman_client_id_ = annotated_message.sender;
@@ -121,18 +121,16 @@ void Shiftboss::run() {
                                             kWorkOrderMessage);
 
         const size_t worker_index = getSchedulableWorker();
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
-                  << "') from Foreman to worker " << worker_index;
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
+                   << "') from Foreman to worker " << worker_index;
 
         const MessageBus::SendStatus send_status =
             QueryExecutionUtil::SendTMBMessage(bus_,
                                                shiftboss_client_id_,
                                                workers_->getClientID(worker_index),
                                                move(worker_tagged_message));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         break;
       }
       case kInitiateRebuildMessage: {
@@ -153,9 +151,10 @@ void Shiftboss::run() {
       case kRebuildWorkOrderCompleteMessage:
       case kDataPipelineMessage:
       case kWorkOrderFeedbackMessage: {
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
-                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') forwarded typed '" << annotated_message.tagged_message.message_type()
+                   << "' message from Worker with TMB client ID '" << annotated_message.sender
+                   << "' to Foreman with TMB client ID " << foreman_client_id_;
 
         DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
         const MessageBus::SendStatus send_status =
@@ -163,9 +162,7 @@ void Shiftboss::run() {
                                                shiftboss_client_id_,
                                                foreman_client_id_,
                                                move(annotated_message.tagged_message));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to Foreman with TMB client ID " << foreman_client_id_;
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         break;
       }
       case kSaveQueryResultMessage: {
@@ -190,23 +187,21 @@ void Shiftboss::run() {
                                        kSaveQueryResultResponseMessage);
         free(proto_response_bytes);
 
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
-                  << "') to Foreman";
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
+                   << "') to Foreman with TMB client ID " << foreman_client_id_;
         const MessageBus::SendStatus send_status =
             QueryExecutionUtil::SendTMBMessage(bus_,
                                                shiftboss_client_id_,
                                                foreman_client_id_,
                                                move(message_response));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to Foreman with TMB client ID " << foreman_client_id_;
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         break;
       }
       case kPoisonMessage: {
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
-                  << "') from Foreman to all workers";
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') forwarded PoisonMessage (typed '" << kPoisonMessage
+                   << "') from Foreman to all workers";
 
         tmb::MessageStyle broadcast_style;
         broadcast_style.Broadcast(true);
@@ -216,9 +211,7 @@ void Shiftboss::run() {
                        worker_addresses_,
                        broadcast_style,
                        move(annotated_message.tagged_message));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be broadcast from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to All workers";
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         return;
       }
       default: {
@@ -245,10 +238,6 @@ size_t Shiftboss::getSchedulableWorker() {
 }
 
 void Shiftboss::registerWithForeman() {
-  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
-            << "') to all";
-
   tmb::Address all_addresses;
   all_addresses.All(true);
 
@@ -266,6 +255,9 @@ void Shiftboss::registerWithForeman() {
                         kShiftbossRegistrationMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+             << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
+             << "') to all";
   tmb::MessageBus::SendStatus send_status =
       bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
   DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
@@ -285,10 +277,6 @@ void Shiftboss::processQueryInitiateMessage(
                        bus_));
   query_contexts_.emplace(query_id, move(query_context));
 
-  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-            << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
-            << "') to Foreman";
-
   serialization::QueryInitiateResponseMessage proto;
   proto.set_query_id(query_id);
 
@@ -301,14 +289,15 @@ void Shiftboss::processQueryInitiateMessage(
                                  kQueryInitiateResponseMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+             << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
+             << "') to Foreman with TMB client ID " << foreman_client_id_;
   const MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          shiftboss_client_id_,
                                          foreman_client_id_,
                                          move(message_response));
-  CHECK(send_status == MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-      << " to Foreman with TMB client ID " << foreman_client_id_;
+  CHECK(send_status == MessageBus::SendStatus::kOK);
 }
 
 void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -324,10 +313,6 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   vector<MutableBlockReference> partially_filled_block_refs;
   insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
 
-  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-            << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
-            << "') to Foreman";
-
   serialization::InitiateRebuildResponseMessage proto;
   proto.set_query_id(query_id);
   proto.set_operator_index(op_index);
@@ -343,14 +328,15 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                                  kInitiateRebuildResponseMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+             << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
+             << "') to Foreman with TMB client ID " << foreman_client_id_;
   const MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          shiftboss_client_id_,
                                          foreman_client_id_,
                                          move(message_response));
-  CHECK(send_status == MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-      << " to Foreman with TMB client ID " << foreman_client_id_;
+  CHECK(send_status == MessageBus::SendStatus::kOK);
 
   for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
     // NOTE(zuyu): Worker releases the memory after the execution of
@@ -371,18 +357,16 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                                         kRebuildWorkOrderMessage);
 
     const size_t worker_index = getSchedulableWorker();
-    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-              << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
-              << "') to worker " << worker_index;
+    DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+               << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
+               << "') to worker " << worker_index;
 
     const MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            shiftboss_client_id_,
                                            workers_->getClientID(worker_index),
                                            move(worker_tagged_message));
-    CHECK(send_status == MessageBus::SendStatus::kOK)
-        << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-        << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
+    CHECK(send_status == MessageBus::SendStatus::kOK);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index d497be6..0b1efba 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -101,12 +101,12 @@ void Worker::sendWorkOrderCompleteMessage(const tmb::client_id receiver,
       static_cast<const void *>(proto_bytes), proto_length, message_type);
   std::free(proto_bytes);
 
+  DLOG(INFO) << "Worker sent WorkOrderCompleteMessage (typed '" << message_type
+             << "') to Scheduler with TMB client ID " << receiver;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(
           bus_, worker_client_id_, receiver, std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "Message could not be sent from worker with TMB client ID "
-      << worker_client_id_ << " to Foreman with TMB client ID " << receiver;
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 template <typename CompletionMessageProtoT>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 8197aef..24da9bf 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -146,17 +146,15 @@ void DeleteWorkOrder::execute() {
                                     kDataPipelineMessage);
   std::free(proto_bytes);
 
-  const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
+  DLOG(INFO) << "DeleteWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(
           bus_,
-          worker_thread_client_id,
+          ClientIDMap::Instance()->getValue(),
           scheduler_client_id_,
           std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not"
-      " be sent from thread with TMB client ID " <<
-      worker_thread_client_id << " to Foreman with TMB client ID "
-      << scheduler_client_id_;
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index fe4be68..2cef1f1 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -101,15 +101,14 @@ class RebuildWorkOrder : public WorkOrder {
 
     // Refer to InsertDestination::sendBlockFilledMessage for the rationale
     // behind using the ClientIDMap map.
+    DLOG(INFO) << "RebuildWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+               << "') to Scheduler with TMB client ID " << scheduler_client_id_;
     const tmb::MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            ClientIDMap::Instance()->getValue(),
                                            scheduler_client_id_,
                                            std::move(tagged_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could "
-        " not be sent from thread with TMB client ID " <<
-        ClientIDMap::Instance()->getValue() << " to Foreman with TMB client ID "
-        << scheduler_client_id_;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index bc29365..143c741 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -128,17 +128,15 @@ void UpdateWorkOrder::execute() {
                                     kDataPipelineMessage);
   std::free(proto_bytes);
 
-  const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
+  DLOG(INFO) << "UpdateWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(
           bus_,
-          worker_thread_client_id,
+          ClientIDMap::Instance()->getValue(),
           scheduler_client_id_,
           std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not"
-      " be sent from thread with TMB client ID " <<
-      worker_thread_client_id << " to Foreman with TMB client ID "
-      << scheduler_client_id_;
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 3cbab94..c1b9b68 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -44,7 +44,6 @@ namespace quickstep {
  *  @{
  */
 
-
 /**
  * @brief A single unit of work in a query plan, produced by a
  *        RelationalOperator. Where possible, WorkOrders should be of
@@ -284,14 +283,14 @@ class WorkOrder {
     tmb::MessageStyle single_receiver_style;
 
     DCHECK(bus != nullptr);
+    DLOG(INFO) << "WorkOrder sent WorkOrderFeedbackMessage (typed '" << kWorkOrderFeedbackMessage
+               << "') to Scheduler with TMB client ID " << receiver_id;
     const tmb::MessageBus::SendStatus send_status =
         bus->Send(sender_id,
                   receiver_address,
                   single_receiver_style,
                   std::move(msg));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could"
-        " not be sent from thread with TMB client ID " << sender_id << " to"
-        " receiver thread with TMB client ID " << receiver_id;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 9897aed..5e83453 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -282,13 +282,15 @@ MutableBlockReference AlwaysCreateBlockInsertDestination::createNewBlock() {
                            kCatalogRelationNewBlockMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "AlwaysCreateBlockInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+             << kCatalogRelationNewBlockMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          thread_id_map_.getValue(),
                                          scheduler_client_id_,
                                          move(tagged_msg));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   return storage_manager_->getBlockMutable(new_id, relation_);
 }
@@ -330,13 +332,15 @@ MutableBlockReference BlockPoolInsertDestination::createNewBlock() {
                            kCatalogRelationNewBlockMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "BlockPoolInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+             << kCatalogRelationNewBlockMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          thread_id_map_.getValue(),
                                          scheduler_client_id_,
                                          move(tagged_msg));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   return storage_manager_->getBlockMutable(new_id, relation_);
 }
@@ -445,13 +449,15 @@ MutableBlockReference PartitionAwareInsertDestination::createNewBlockInPartition
                            kCatalogRelationNewBlockMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "PartitionAwareInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+             << kCatalogRelationNewBlockMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          thread_id_map_.getValue(),
                                          scheduler_client_id_,
                                          move(tagged_msg));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   return storage_manager_->getBlockMutable(new_id, relation_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 3dae9a0..408e76b 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -249,16 +249,14 @@ class InsertDestination : public InsertDestinationInterface {
     // option 3.
     DCHECK(bus_ != nullptr);
 
-    const tmb::client_id worker_thread_client_id = thread_id_map_.getValue();
+    DLOG(INFO) << "InsertDestination sent DataPipelineMessage (typed '" << kDataPipelineMessage
+               << "') to Scheduler with TMB client ID " << scheduler_client_id_;
     const tmb::MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
-                                           worker_thread_client_id,
+                                           thread_id_map_.getValue(),
                                            scheduler_client_id_,
                                            std::move(tagged_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-        "Message could not be sent from thread with TMB client ID "
-        << worker_thread_client_id << " to Scheduler with TMB client"
-        " ID " << scheduler_client_id_;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
   inline const std::size_t getQueryID() const {


[16/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/DeadLockDetector.cpp
----------------------------------------------------------------------
diff --git a/transaction/DeadLockDetector.cpp b/transaction/DeadLockDetector.cpp
index 26ab115..f213e1a 100644
--- a/transaction/DeadLockDetector.cpp
+++ b/transaction/DeadLockDetector.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/DeadLockDetector.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/DeadLockDetector.hpp
----------------------------------------------------------------------
diff --git a/transaction/DeadLockDetector.hpp b/transaction/DeadLockDetector.hpp
index 6897afb..08a0c25 100644
--- a/transaction/DeadLockDetector.hpp
+++ b/transaction/DeadLockDetector.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/DirectedGraph.hpp
----------------------------------------------------------------------
diff --git a/transaction/DirectedGraph.hpp b/transaction/DirectedGraph.hpp
index 16b551a..ad209ac 100644
--- a/transaction/DirectedGraph.hpp
+++ b/transaction/DirectedGraph.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_DIRECTED_GRAPH_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/Lock.hpp
----------------------------------------------------------------------
diff --git a/transaction/Lock.hpp b/transaction/Lock.hpp
index 27fe93a..7c2a23c 100644
--- a/transaction/Lock.hpp
+++ b/transaction/Lock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_LOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/LockManager.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.cpp b/transaction/LockManager.cpp
index da6181a..2a3760f 100644
--- a/transaction/LockManager.cpp
+++ b/transaction/LockManager.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/LockManager.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/LockManager.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.hpp b/transaction/LockManager.hpp
index 40ee6c8..6606dc2 100644
--- a/transaction/LockManager.hpp
+++ b/transaction/LockManager.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/LockRequest.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockRequest.hpp b/transaction/LockRequest.hpp
index 80daa4d..e31a68d 100644
--- a/transaction/LockRequest.hpp
+++ b/transaction/LockRequest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_LOCK_REQUEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/LockTable.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockTable.cpp b/transaction/LockTable.cpp
index 7b568fc..72e735f 100644
--- a/transaction/LockTable.cpp
+++ b/transaction/LockTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/LockTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/LockTable.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockTable.hpp b/transaction/LockTable.hpp
index 529db12..c547a64 100644
--- a/transaction/LockTable.hpp
+++ b/transaction/LockTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_LOCK_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/ResourceId.cpp
----------------------------------------------------------------------
diff --git a/transaction/ResourceId.cpp b/transaction/ResourceId.cpp
index bae92f8..fe967b5 100644
--- a/transaction/ResourceId.cpp
+++ b/transaction/ResourceId.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/ResourceId.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/ResourceId.hpp
----------------------------------------------------------------------
diff --git a/transaction/ResourceId.hpp b/transaction/ResourceId.hpp
index 3a770dd..e89c012 100644
--- a/transaction/ResourceId.hpp
+++ b/transaction/ResourceId.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_RESOURCE_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/StronglyConnectedComponents.cpp
----------------------------------------------------------------------
diff --git a/transaction/StronglyConnectedComponents.cpp b/transaction/StronglyConnectedComponents.cpp
index f50ed85..335a99d 100644
--- a/transaction/StronglyConnectedComponents.cpp
+++ b/transaction/StronglyConnectedComponents.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/StronglyConnectedComponents.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/StronglyConnectedComponents.hpp
----------------------------------------------------------------------
diff --git a/transaction/StronglyConnectedComponents.hpp b/transaction/StronglyConnectedComponents.hpp
index ca58a68..3fa5be0 100644
--- a/transaction/StronglyConnectedComponents.hpp
+++ b/transaction/StronglyConnectedComponents.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_STRONGLY_CONNECTED_COMPONENTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/Transaction.hpp
----------------------------------------------------------------------
diff --git a/transaction/Transaction.hpp b/transaction/Transaction.hpp
index 69d4311..c156426 100644
--- a/transaction/Transaction.hpp
+++ b/transaction/Transaction.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_TRANSACTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/TransactionModule.hpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionModule.hpp b/transaction/TransactionModule.hpp
index 46e7d15..d0dc0e5 100644
--- a/transaction/TransactionModule.hpp
+++ b/transaction/TransactionModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/TransactionTable.cpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionTable.cpp b/transaction/TransactionTable.cpp
index 3e37439..8071374 100644
--- a/transaction/TransactionTable.cpp
+++ b/transaction/TransactionTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/TransactionTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/TransactionTable.hpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionTable.hpp b/transaction/TransactionTable.hpp
index 29df536..b3fae53 100644
--- a/transaction/TransactionTable.hpp
+++ b/transaction/TransactionTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TRANSACTION_TRANSACTION_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/AccessMode_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/AccessMode_unittest.cpp b/transaction/tests/AccessMode_unittest.cpp
index 3287fb0..953e677 100644
--- a/transaction/tests/AccessMode_unittest.cpp
+++ b/transaction/tests/AccessMode_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/AccessMode.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/CycleDetector_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/CycleDetector_unittest.cpp b/transaction/tests/CycleDetector_unittest.cpp
index 6edaa63..201dcdf 100644
--- a/transaction/tests/CycleDetector_unittest.cpp
+++ b/transaction/tests/CycleDetector_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/CycleDetector.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/DeadLockDetector_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/DeadLockDetector_unittest.cpp b/transaction/tests/DeadLockDetector_unittest.cpp
index bc65ef5..a659214 100644
--- a/transaction/tests/DeadLockDetector_unittest.cpp
+++ b/transaction/tests/DeadLockDetector_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/DeadLockDetector.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/DirectedGraph_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/DirectedGraph_unittest.cpp b/transaction/tests/DirectedGraph_unittest.cpp
index 00fe276..c40ac35 100644
--- a/transaction/tests/DirectedGraph_unittest.cpp
+++ b/transaction/tests/DirectedGraph_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/DirectedGraph.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/LockRequest_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/LockRequest_unittest.cpp b/transaction/tests/LockRequest_unittest.cpp
index 77047d9..94ccc32 100644
--- a/transaction/tests/LockRequest_unittest.cpp
+++ b/transaction/tests/LockRequest_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/LockRequest.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/LockTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/LockTable_unittest.cpp b/transaction/tests/LockTable_unittest.cpp
index 1aed0b8..961065b 100644
--- a/transaction/tests/LockTable_unittest.cpp
+++ b/transaction/tests/LockTable_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/LockTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/Lock_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/Lock_unittest.cpp b/transaction/tests/Lock_unittest.cpp
index 59a5e7a..9cad301 100644
--- a/transaction/tests/Lock_unittest.cpp
+++ b/transaction/tests/Lock_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/Lock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/ResourceId_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/ResourceId_unittest.cpp b/transaction/tests/ResourceId_unittest.cpp
index f3029f1..8292cb2 100644
--- a/transaction/tests/ResourceId_unittest.cpp
+++ b/transaction/tests/ResourceId_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/ResourceId.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/StronglyConnectedComponents_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/StronglyConnectedComponents_unittest.cpp b/transaction/tests/StronglyConnectedComponents_unittest.cpp
index 35ef842..e9bc3a3 100644
--- a/transaction/tests/StronglyConnectedComponents_unittest.cpp
+++ b/transaction/tests/StronglyConnectedComponents_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/StronglyConnectedComponents.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/transaction/tests/TransactionTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/TransactionTable_unittest.cpp b/transaction/tests/TransactionTable_unittest.cpp
index cd47a2b..0a852ea 100644
--- a/transaction/tests/TransactionTable_unittest.cpp
+++ b/transaction/tests/TransactionTable_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "transaction/TransactionTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/CMakeLists.txt b/types/CMakeLists.txt
index 420891a..769187b 100644
--- a/types/CMakeLists.txt
+++ b/types/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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(CheckTypeSize)
 set(CMAKE_EXTRA_INCLUDE_FILES "time.h")

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/CharType.cpp
----------------------------------------------------------------------
diff --git a/types/CharType.cpp b/types/CharType.cpp
index ef96150..591c038 100644
--- a/types/CharType.cpp
+++ b/types/CharType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/CharType.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index 5c241f3..c7321f4 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_CHAR_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DateOperatorOverloads.hpp
----------------------------------------------------------------------
diff --git a/types/DateOperatorOverloads.hpp b/types/DateOperatorOverloads.hpp
index 7109ae1..5cf8990 100644
--- a/types/DateOperatorOverloads.hpp
+++ b/types/DateOperatorOverloads.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_DATE_BINARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DateType.cpp
----------------------------------------------------------------------
diff --git a/types/DateType.cpp b/types/DateType.cpp
index 9637acc..5bb982c 100644
--- a/types/DateType.cpp
+++ b/types/DateType.cpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DateType.hpp
----------------------------------------------------------------------
diff --git a/types/DateType.hpp b/types/DateType.hpp
index 2a13be1..07225d5 100644
--- a/types/DateType.hpp
+++ b/types/DateType.hpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 QUICKSTEP_TYPES_DATE_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DatetimeIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.cpp b/types/DatetimeIntervalType.cpp
index 3054ccb..1eae03a 100644
--- a/types/DatetimeIntervalType.cpp
+++ b/types/DatetimeIntervalType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DatetimeIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.hpp b/types/DatetimeIntervalType.hpp
index a14396d..005cb31 100644
--- a/types/DatetimeIntervalType.hpp
+++ b/types/DatetimeIntervalType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_DATETIME_INTERVAL_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DatetimeLit.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeLit.hpp b/types/DatetimeLit.hpp
index c87b510..580a0ed 100644
--- a/types/DatetimeLit.hpp
+++ b/types/DatetimeLit.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_DATETIME_LIT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DatetimeType.cpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.cpp b/types/DatetimeType.cpp
index d19ebd2..f54f318 100644
--- a/types/DatetimeType.cpp
+++ b/types/DatetimeType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DatetimeType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.hpp b/types/DatetimeType.hpp
index 0e28daa..aad536a 100644
--- a/types/DatetimeType.hpp
+++ b/types/DatetimeType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_DATETIME_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DoubleType.cpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.cpp b/types/DoubleType.cpp
index 642430f..6a7914c 100644
--- a/types/DoubleType.cpp
+++ b/types/DoubleType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/DoubleType.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/DoubleType.hpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.hpp b/types/DoubleType.hpp
index 3dc4fe6..b4175b0 100644
--- a/types/DoubleType.hpp
+++ b/types/DoubleType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_DOUBLE_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/FloatType.cpp
----------------------------------------------------------------------
diff --git a/types/FloatType.cpp b/types/FloatType.cpp
index 01bf4bf..e904f29 100644
--- a/types/FloatType.cpp
+++ b/types/FloatType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/FloatType.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/FloatType.hpp
----------------------------------------------------------------------
diff --git a/types/FloatType.hpp b/types/FloatType.hpp
index ffd34e8..2a156e1 100644
--- a/types/FloatType.hpp
+++ b/types/FloatType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_FLOAT_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/IntType.cpp
----------------------------------------------------------------------
diff --git a/types/IntType.cpp b/types/IntType.cpp
index 512af10..9781675 100644
--- a/types/IntType.cpp
+++ b/types/IntType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/IntType.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/IntType.hpp
----------------------------------------------------------------------
diff --git a/types/IntType.hpp b/types/IntType.hpp
index 4509aee..08d6b3d 100644
--- a/types/IntType.hpp
+++ b/types/IntType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_INT_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/IntervalLit.hpp
----------------------------------------------------------------------
diff --git a/types/IntervalLit.hpp b/types/IntervalLit.hpp
index 940d6b8..8b1ae28 100644
--- a/types/IntervalLit.hpp
+++ b/types/IntervalLit.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_INTERVAL_LIT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/IntervalParser.cpp
----------------------------------------------------------------------
diff --git a/types/IntervalParser.cpp b/types/IntervalParser.cpp
index 92694d5..b033e6a 100644
--- a/types/IntervalParser.cpp
+++ b/types/IntervalParser.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS



[43/50] [abbrv] incubator-quickstep git commit: Deserialized Window Aggr WorkOrder.

Posted by zu...@apache.org.
Deserialized Window Aggr WorkOrder.


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

Branch: refs/heads/travis-grpc
Commit: 85e02de49205409accfef3737dadfe95aad1f5c0
Parents: 658cb61
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 23:14:08 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 23:14:08 2016 -0700

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt       |  1 +
 relational_operators/WorkOrderFactory.cpp | 25 +++++++++++++++++++++++++
 2 files changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85e02de4/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 9696392..43a42f9 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -467,6 +467,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
                       quickstep_relationaloperators_UpdateOperator
+                      quickstep_relationaloperators_WindowAggregationOperator
                       quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85e02de4/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index f920cac..721d735 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -46,6 +46,7 @@
 #include "relational_operators/TableGeneratorOperator.hpp"
 #include "relational_operators/TextScanOperator.hpp"
 #include "relational_operators/UpdateOperator.hpp"
+#include "relational_operators/WindowAggregationOperator.hpp"
 #include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 
@@ -419,6 +420,22 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           shiftboss_client_id,
           bus);
     }
+    case serialization::WINDOW_AGGREGATION: {
+      LOG(INFO) << "Creating WindowAggregationWorkOrder";
+      vector<block_id> blocks;
+      for (int i = 0; i < proto.ExtensionSize(serialization::WindowAggregationWorkOrder::block_ids); ++i) {
+        blocks.push_back(
+            proto.GetExtension(serialization::WindowAggregationWorkOrder::block_ids, i));
+      }
+
+      return new WindowAggregationWorkOrder(
+          proto.query_id(),
+          query_context->getWindowAggregationState(
+              proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)),
+          move(blocks),
+          query_context->getInsertDestination(
+              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
+    }
     default:
       LOG(FATAL) << "Unknown WorkOrder Type in WorkOrderFactory::ReconstructFromProto";
   }
@@ -697,6 +714,14 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              proto.HasExtension(serialization::UpdateWorkOrder::operator_index) &&
              proto.HasExtension(serialization::UpdateWorkOrder::block_id);
     }
+    case serialization::WINDOW_AGGREGATION: {
+      return proto.HasExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index) &&
+             query_context.isValidWindowAggregationStateId(
+                 proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)) &&
+             proto.HasExtension(serialization::WindowAggregationWorkOrder::insert_destination_index) &&
+             query_context.isValidInsertDestinationId(
+                 proto.GetExtension(serialization::WindowAggregationWorkOrder::insert_destination_index));
+    }
     default:
       return false;
   }


[05/50] [abbrv] incubator-quickstep git commit: Removed the redundant query id in the optimizer.

Posted by zu...@apache.org.
Removed the redundant query id in the optimizer.


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

Branch: refs/heads/travis-grpc
Commit: 8cd5a56c92f2e0e6c2acb2a979a2ae5fb2e54792
Parents: d4e714c
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 1 23:02:37 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Wed Aug 3 11:53:46 2016 -0700

----------------------------------------------------------------------
 cli/tests/CommandExecutorTestRunner.cpp         |  5 ++---
 query_optimizer/CMakeLists.txt                  |  3 ---
 query_optimizer/ExecutionGenerator.cpp          |  4 +++-
 query_optimizer/Optimizer.hpp                   |  9 ++-------
 query_optimizer/OptimizerContext.hpp            | 20 ++------------------
 query_optimizer/QueryProcessor.cpp              | 10 ++--------
 query_optimizer/QueryProcessor.hpp              |  7 +++++++
 .../tests/ExecutionGeneratorTestRunner.cpp      |  5 ++---
 query_optimizer/tests/OptimizerTest.cpp         |  3 +--
 .../tests/OptimizerTextTestRunner.cpp           |  3 +--
 10 files changed, 22 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/cli/tests/CommandExecutorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.cpp b/cli/tests/CommandExecutorTestRunner.cpp
index bd7082f..dc7e43f 100644
--- a/cli/tests/CommandExecutorTestRunner.cpp
+++ b/cli/tests/CommandExecutorTestRunner.cpp
@@ -76,8 +76,7 @@ void CommandExecutorTestRunner::runTestCase(
   while (true) {
     ParseResult result = sql_parser_.getNextStatement();
 
-    O::OptimizerContext optimizer_context(0 /* query_id */,
-                                          test_database_loader_.catalog_database(),
+    O::OptimizerContext optimizer_context(test_database_loader_.catalog_database(),
                                           test_database_loader_.storage_manager());
 
     if (result.condition != ParseResult::kSuccess) {
@@ -99,7 +98,7 @@ void CommandExecutorTestRunner::runTestCase(
               nullptr,
               output_stream.file());
         } else  {
-          QueryHandle query_handle(optimizer_context.query_id());
+          QueryHandle query_handle(0 /* query_id */);
           O::LogicalGenerator logical_generator(&optimizer_context);
           O::PhysicalGenerator physical_generator;
           O::ExecutionGenerator execution_generator(&optimizer_context, &query_handle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index c55881f..04e54d3 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -185,7 +185,6 @@ target_link_libraries(quickstep_queryoptimizer_Optimizer
                       quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_OptimizerContext
-                      quickstep_catalog_CatalogTypedefs
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_OptimizerTree
@@ -219,8 +218,6 @@ target_link_libraries(quickstep_queryoptimizer_QueryPlan
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_QueryProcessor
                       quickstep_catalog_Catalog
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_CatalogRelation
                       quickstep_catalog_Catalog_proto
                       quickstep_parser_ParseStatement
                       quickstep_queryoptimizer_Optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index fb24489..2ae6a4b 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -26,6 +26,8 @@
 #include <type_traits>
 #include <unordered_map>
 
+#include "query_optimizer/QueryOptimizerConfig.h"  // For QUICKSTEP_DISTRIBUTED.
+
 #ifdef QUICKSTEP_DISTRIBUTED
 #include <unordered_set>
 #endif
@@ -299,7 +301,7 @@ void ExecutionGenerator::generatePlanInternal(
 std::string ExecutionGenerator::getNewRelationName() {
   std::ostringstream out;
   out << OptimizerContext::kInternalTemporaryRelationNamePrefix
-      << optimizer_context_->query_id() << "_" << rel_id_;
+      << query_handle_->query_id() << "_" << rel_id_;
   ++rel_id_;
   return out.str();
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/Optimizer.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.hpp b/query_optimizer/Optimizer.hpp
index 9177414..18c927d 100644
--- a/query_optimizer/Optimizer.hpp
+++ b/query_optimizer/Optimizer.hpp
@@ -18,8 +18,6 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_HPP_
 
-#include <cstddef>
-
 #include "query_optimizer/OptimizerContext.hpp"
 #include "utility/Macros.hpp"
 
@@ -45,15 +43,12 @@ class Optimizer {
   /**
    * @brief Constructor.
    *
-   * @param query_id The query id. Used to identify a query and create distinct
-   *        names for temporary relations.
    * @param database The database that the query is executed on.
    * @param storage_manager The storage manager for the database.
    */
-  Optimizer(const std::size_t query_id,
-            CatalogDatabase *database,
+  Optimizer(CatalogDatabase *database,
             StorageManager *storage_manager)
-      : optimizer_context_(query_id, database, storage_manager) {}
+      : optimizer_context_(database, storage_manager) {}
 
   /**
    * @brief Destructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/OptimizerContext.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerContext.hpp b/query_optimizer/OptimizerContext.hpp
index abdc7f4..bcab9d0 100644
--- a/query_optimizer/OptimizerContext.hpp
+++ b/query_optimizer/OptimizerContext.hpp
@@ -18,18 +18,12 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_CONTEXT_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_OPTIMIZER_CONTEXT_HPP_
 
-#include <cstddef>
-#include <cstdlib>
-#include <string>
-
-#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class CatalogDatabase;
-class CatalogRelation;
 class StorageManager;
 
 namespace optimizer {
@@ -49,16 +43,13 @@ class OptimizerContext {
   /**
    * @brief Constructor.
    *
-   * @param query_id The query ID.
    * @param catalog_database The catalog database where this query is executed.
    * @param storage_manager The storage manager to use for allocating storage
    *        blocks.
    */
-  OptimizerContext(const std::size_t query_id,
-                   CatalogDatabase *catalog_database,
+  OptimizerContext(CatalogDatabase *catalog_database,
                    StorageManager *storage_manager)
-      : query_id_(query_id),
-        current_expr_id_(-1),
+      : current_expr_id_(-1),
         catalog_database_(catalog_database),
         storage_manager_(storage_manager),
         has_nested_queries_(false),
@@ -94,11 +85,6 @@ class OptimizerContext {
   StorageManager* storage_manager() { return storage_manager_; }
 
   /**
-   * @return Query ID.
-   */
-  std::size_t query_id() const { return query_id_; }
-
-  /**
    * @brief Gets the next ExprId.
    *
    * @return A new ExprId.
@@ -133,8 +119,6 @@ class OptimizerContext {
   bool is_catalog_changed() const { return is_catalog_changed_; }
 
  private:
-  const std::size_t query_id_;
-
   expressions::ExprId current_expr_id_;
 
   CatalogDatabase *catalog_database_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/QueryProcessor.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.cpp b/query_optimizer/QueryProcessor.cpp
index 6381d3a..8af4408 100644
--- a/query_optimizer/QueryProcessor.cpp
+++ b/query_optimizer/QueryProcessor.cpp
@@ -17,20 +17,15 @@
 
 #include "query_optimizer/QueryProcessor.hpp"
 
-#include <cstdint>
-#include <cstdlib>
 #include <fstream>
 #include <memory>
 #include <string>
 
 #include "catalog/Catalog.hpp"
 #include "catalog/Catalog.pb.h"
-#include "catalog/CatalogDatabase.hpp"
-#include "catalog/CatalogRelation.hpp"
 #include "parser/ParseStatement.hpp"
 #include "query_optimizer/Optimizer.hpp"
 #include "query_optimizer/QueryHandle.hpp"
-#include "storage/StorageManager.hpp"
 
 using std::ifstream;
 using std::ofstream;
@@ -41,10 +36,9 @@ QueryHandle* QueryProcessor::generateQueryHandle(const ParseStatement &statement
   std::unique_ptr<QueryHandle> query_handle(
       new QueryHandle(query_id_, statement.getPriority()));
 
-  optimizer::Optimizer optimizer(query_id_, getDefaultDatabase(), storage_manager_.get());
-  optimizer.generateQueryHandle(statement, query_handle.get());
+  optimizer_->generateQueryHandle(statement, query_handle.get());
 
-  if (optimizer.isCatalogChanged() && !catalog_altered_) {
+  if (optimizer_->isCatalogChanged() && !catalog_altered_) {
     catalog_altered_ = true;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/QueryProcessor.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.hpp b/query_optimizer/QueryProcessor.hpp
index 32739dc..f3844a0 100644
--- a/query_optimizer/QueryProcessor.hpp
+++ b/query_optimizer/QueryProcessor.hpp
@@ -26,6 +26,7 @@
 #include <string>
 
 #include "catalog/Catalog.hpp"
+#include "query_optimizer/Optimizer.hpp"
 #include "storage/StorageManager.hpp"
 #include "utility/Macros.hpp"
 
@@ -139,6 +140,9 @@ class QueryProcessor {
         query_id_(0) {
     loadCatalog();
     storage_manager_.reset(new StorageManager(storage_path));
+
+    // Construct after Catalog loads and StorageManager initializes.
+    optimizer_.reset(new optimizer::Optimizer(getDefaultDatabase(), storage_manager_.get()));
   }
 
   /**
@@ -190,6 +194,9 @@ class QueryProcessor {
 
   std::unique_ptr<Catalog> catalog_;
   std::unique_ptr<StorageManager> storage_manager_;
+
+  std::unique_ptr<optimizer::Optimizer> optimizer_;
+
   bool catalog_altered_;
 
   std::size_t query_id_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index 563a777..fd1bb86 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -73,8 +73,7 @@ void ExecutionGeneratorTestRunner::runTestCase(
   while (true) {
     ParseResult result = sql_parser_.getNextStatement();
 
-    OptimizerContext optimizer_context(0 /* query_id */,
-                                       test_database_loader_.catalog_database(),
+    OptimizerContext optimizer_context(test_database_loader_.catalog_database(),
                                        test_database_loader_.storage_manager());
 
     if (result.condition != ParseResult::kSuccess) {
@@ -85,7 +84,7 @@ void ExecutionGeneratorTestRunner::runTestCase(
     } else {
       std::printf("%s\n", result.parsed_statement->toString().c_str());
       try {
-        QueryHandle query_handle(optimizer_context.query_id());
+        QueryHandle query_handle(0 /* query_id */);
         LogicalGenerator logical_generator(&optimizer_context);
         PhysicalGenerator physical_generator;
         ExecutionGenerator execution_generator(&optimizer_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/tests/OptimizerTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTest.cpp b/query_optimizer/tests/OptimizerTest.cpp
index 3734719..57e2d67 100644
--- a/query_optimizer/tests/OptimizerTest.cpp
+++ b/query_optimizer/tests/OptimizerTest.cpp
@@ -59,8 +59,7 @@ OptimizerTest::OptimizerTest()
     : catalog_(new Catalog),
       catalog_database_(
           new CatalogDatabase(catalog_.get(), "TestDatabase" /* name */, 0)),
-      optimizer_context_(new OptimizerContext(0 /* query_id */,
-                                              catalog_database_.get(),
+      optimizer_context_(new OptimizerContext(catalog_database_.get(),
                                               nullptr /* storage_manager */)),
       physical_generator_(new PhysicalGenerator()) {}
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8cd5a56c/query_optimizer/tests/OptimizerTextTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.cpp b/query_optimizer/tests/OptimizerTextTestRunner.cpp
index 251b64b..d790c33 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.cpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.cpp
@@ -47,8 +47,7 @@ void OptimizerTextTestRunner::runTestCase(const std::string &input,
   sql_parser_.feedNextBuffer(new std::string(input));
   ParseResult result = sql_parser_.getNextStatement();
 
-  OptimizerContext optimizer_context(0 /* query_id */,
-                                     test_database_loader_.catalog_database(),
+  OptimizerContext optimizer_context(test_database_loader_.catalog_database(),
                                      nullptr /* storage_manager */);
   if (result.condition != ParseResult::kSuccess) {
     *output = result.error_message;


[41/50] [abbrv] incubator-quickstep git commit: Added PolicyEnforcer implementation for the distributed version.

Posted by zu...@apache.org.
Added PolicyEnforcer implementation for the distributed version.


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

Branch: refs/heads/travis-grpc
Commit: 3c2749eafcff25283915bdef21822056d36f5281
Parents: e443b2b
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Thu Aug 4 11:45:51 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:35:45 2016 -0700

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                |  24 ++
 query_execution/PolicyEnforcerBase.cpp        |   2 +
 query_execution/PolicyEnforcerBase.hpp        |   7 +
 query_execution/PolicyEnforcerDistributed.cpp | 279 +++++++++++++++++++++
 query_execution/PolicyEnforcerDistributed.hpp | 113 +++++++++
 query_execution/QueryExecutionMessages.proto  |  16 +-
 query_execution/QueryExecutionTypedefs.hpp    |   5 +
 query_execution/QueryManagerBase.cpp          |   3 +-
 query_execution/QueryManagerBase.hpp          |  11 +-
 9 files changed, 456 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 4b180e3..74fcafb 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -35,6 +35,9 @@ endif()
 add_library(quickstep_queryexecution_ForemanBase ../empty_src.cpp ForemanBase.hpp)
 add_library(quickstep_queryexecution_ForemanSingleNode ForemanSingleNode.cpp ForemanSingleNode.hpp)
 add_library(quickstep_queryexecution_PolicyEnforcerBase PolicyEnforcerBase.cpp PolicyEnforcerBase.hpp)
+if (ENABLE_DISTRIBUTED)
+  add_library(quickstep_queryexecution_PolicyEnforcerDistributed PolicyEnforcerDistributed.cpp PolicyEnforcerDistributed.hpp)
+endif(ENABLE_DISTRIBUTED)
 add_library(quickstep_queryexecution_PolicyEnforcerSingleNode PolicyEnforcerSingleNode.cpp PolicyEnforcerSingleNode.hpp)
 add_library(quickstep_queryexecution_QueryContext QueryContext.cpp QueryContext.hpp)
 add_library(quickstep_queryexecution_QueryContext_proto
@@ -110,6 +113,26 @@ target_link_libraries(quickstep_queryexecution_PolicyEnforcerBase
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros
                       tmb)
+if (ENABLE_DISTRIBUTED)
+  target_link_libraries(quickstep_queryexecution_PolicyEnforcerDistributed
+                        glog
+                        quickstep_catalog_CatalogRelation
+                        quickstep_catalog_Catalog_proto
+                        quickstep_queryexecution_PolicyEnforcerBase
+                        quickstep_queryexecution_QueryContext_proto
+                        quickstep_queryexecution_QueryExecutionMessages_proto
+                        quickstep_queryexecution_QueryExecutionState
+                        quickstep_queryexecution_QueryExecutionTypedefs
+                        quickstep_queryexecution_QueryExecutionUtil
+                        quickstep_queryexecution_QueryManagerBase
+                        quickstep_queryexecution_QueryManagerDistributed
+                        quickstep_queryexecution_ShiftbossDirectory
+                        quickstep_queryoptimizer_QueryHandle
+                        quickstep_storage_StorageBlockInfo
+                        quickstep_utility_Macros
+                        tmb
+                        ${GFLAGS_LIB_NAME})
+endif(ENABLE_DISTRIBUTED)
 target_link_libraries(quickstep_queryexecution_PolicyEnforcerSingleNode
                       glog
                       quickstep_catalog_CatalogTypedefs
@@ -293,6 +316,7 @@ target_link_libraries(quickstep_queryexecution
 if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution
                         quickstep_queryexecution_BlockLocator
+                        quickstep_queryexecution_PolicyEnforcerDistributed
                         quickstep_queryexecution_QueryManagerDistributed
                         quickstep_queryexecution_Shiftboss
                         quickstep_queryexecution_ShiftbossDirectory)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index bf6edf9..78f7b44 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -136,6 +136,8 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
   }
   if (admitted_queries_[query_id]->queryStatus(op_index) ==
           QueryManagerBase::QueryStatusCode::kQueryExecuted) {
+    onQueryCompletion(admitted_queries_[query_id].get());
+
     removeQuery(query_id);
     if (!waiting_queries_.empty()) {
       // Admit the earliest waiting query.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.hpp b/query_execution/PolicyEnforcerBase.hpp
index c75a531..e95799e 100644
--- a/query_execution/PolicyEnforcerBase.hpp
+++ b/query_execution/PolicyEnforcerBase.hpp
@@ -140,6 +140,13 @@ class PolicyEnforcerBase {
   static constexpr std::size_t kMaxConcurrentQueries = 1;
 
   /**
+   * @brief Add custom actions upon the completion of a query.
+   *
+   * @param query_manager The query manager.
+   **/
+  virtual void onQueryCompletion(QueryManagerBase *query_manager) {}
+
+  /**
    * @brief Record the execution time for a finished WorkOrder.
    *
    * TODO(harshad) - Extend the functionality to rebuild work orders.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
new file mode 100644
index 0000000..6d0de47
--- /dev/null
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -0,0 +1,279 @@
+/**
+ * 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.
+ **/
+
+#include "query_execution/PolicyEnforcerDistributed.hpp"
+
+#include <cstddef>
+#include <cstdlib>
+#include <memory>
+#include <queue>
+#include <utility>
+#include <unordered_map>
+#include <vector>
+
+#include "catalog/Catalog.pb.h"
+#include "catalog/CatalogRelation.hpp"
+#include "query_execution/QueryContext.pb.h"
+#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/QueryManagerBase.hpp"
+#include "query_execution/QueryManagerDistributed.hpp"
+#include "query_optimizer/QueryHandle.hpp"
+#include "storage/StorageBlockInfo.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+#include "tmb/tagged_message.h"
+
+using std::free;
+using std::malloc;
+using std::move;
+using std::size_t;
+using std::unique_ptr;
+using std::vector;
+
+using tmb::TaggedMessage;
+
+namespace quickstep {
+
+namespace S = serialization;
+
+DEFINE_uint64(max_msgs_per_dispatch_round, 20, "Maximum number of messages that"
+              " can be allocated in a single round of dispatch of messages to"
+              " the workers.");
+
+void PolicyEnforcerDistributed::getWorkOrderMessages(
+    vector<unique_ptr<S::WorkOrderMessage>> *work_order_messages) {
+  // Iterate over admitted queries until either there are no more
+  // messages available, or the maximum number of messages have
+  // been collected.
+  DCHECK(work_order_messages->empty());
+  // TODO(harshad) - Make this function generic enough so that it
+  // works well when multiple queries are getting executed.
+  if (admitted_queries_.empty()) {
+    LOG(WARNING) << "Requesting WorkerMessages when no query is running";
+    return;
+  }
+
+  const std::size_t per_query_share =
+      FLAGS_max_msgs_per_dispatch_round / admitted_queries_.size();
+  DCHECK_GT(per_query_share, 0u);
+
+  vector<std::size_t> finished_queries_ids;
+
+  for (const auto &admitted_query_info : admitted_queries_) {
+    QueryManagerBase *curr_query_manager = admitted_query_info.second.get();
+    DCHECK(curr_query_manager != nullptr);
+    std::size_t messages_collected_curr_query = 0;
+    while (messages_collected_curr_query < per_query_share) {
+      S::WorkOrderMessage *next_work_order_message =
+          static_cast<QueryManagerDistributed*>(curr_query_manager)->getNextWorkOrderMessage(0);
+      if (next_work_order_message != nullptr) {
+        ++messages_collected_curr_query;
+        work_order_messages->push_back(unique_ptr<S::WorkOrderMessage>(next_work_order_message));
+      } else {
+        // No more work ordes from the current query at this time.
+        // Check if the query's execution is over.
+        if (curr_query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+          // If the query has been executed, remove it.
+          finished_queries_ids.push_back(admitted_query_info.first);
+        }
+        break;
+      }
+    }
+  }
+  for (const std::size_t finished_qid : finished_queries_ids) {
+    onQueryCompletion(admitted_queries_[finished_qid].get());
+    removeQuery(finished_qid);
+  }
+}
+
+bool PolicyEnforcerDistributed::admitQuery(QueryHandle *query_handle) {
+  if (admitted_queries_.size() < PolicyEnforcerBase::kMaxConcurrentQueries) {
+    // Ok to admit the query.
+    const std::size_t query_id = query_handle->query_id();
+    if (admitted_queries_.find(query_id) == admitted_queries_.end()) {
+      // NOTE(zuyu): Should call before constructing a 'QueryManager'.
+      // Otherwise, an InitiateRebuildMessage may be sent before 'QueryContext'
+      // initializes.
+      initiateQueryInShiftboss(query_handle);
+
+      // Query with the same ID not present, ok to admit.
+      admitted_queries_[query_id].reset(
+          new QueryManagerDistributed(query_handle, shiftboss_directory_, foreman_client_id_, bus_));
+      return true;
+    } else {
+      LOG(ERROR) << "Query with the same ID " << query_id << " exists";
+      return false;
+    }
+  } else {
+    // This query will have to wait.
+    waiting_queries_.push(query_handle);
+    return false;
+  }
+}
+
+void PolicyEnforcerDistributed::processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message) {
+  S::InitiateRebuildResponseMessage proto;
+  CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+  const std::size_t query_id = proto.query_id();
+  DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+
+  QueryManagerDistributed *query_manager = static_cast<QueryManagerDistributed*>(admitted_queries_[query_id].get());
+
+  const std::size_t num_rebuild_work_orders = proto.num_rebuild_work_orders();
+  query_manager->processInitiateRebuildResponseMessage(proto.operator_index(), num_rebuild_work_orders);
+  shiftboss_directory_->addNumQueuedWorkOrders(proto.shiftboss_index(), num_rebuild_work_orders);
+
+  if (query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+    onQueryCompletion(query_manager);
+
+    removeQuery(query_id);
+    if (!waiting_queries_.empty()) {
+      // Admit the earliest waiting query.
+      QueryHandle *new_query = waiting_queries_.front();
+      waiting_queries_.pop();
+      admitQuery(new_query);
+    }
+  }
+}
+
+void PolicyEnforcerDistributed::initiateQueryInShiftboss(QueryHandle *query_handle) {
+  S::QueryInitiateMessage proto;
+  proto.set_query_id(query_handle->query_id());
+  proto.mutable_catalog_database_cache()->MergeFrom(query_handle->getCatalogDatabaseCacheProto());
+  proto.mutable_query_context()->MergeFrom(query_handle->getQueryContextProto());
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage message(static_cast<const void*>(proto_bytes),
+                        proto_length,
+                        kQueryInitiateMessage);
+  free(proto_bytes);
+
+  LOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
+            << "') to Shiftboss 0";
+
+  // TODO(zuyu): Multiple Shiftbosses support.
+  const tmb::MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         foreman_client_id_,
+                                         shiftboss_directory_->getClientId(0),
+                                         move(message));
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+
+  // Wait Shiftboss for QueryInitiateResponseMessage.
+  const tmb::AnnotatedMessage annotated_message = bus_->Receive(foreman_client_id_, 0, true);
+  const TaggedMessage &tagged_message = annotated_message.tagged_message;
+  DCHECK_EQ(kQueryInitiateResponseMessage, tagged_message.message_type());
+  LOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
+            << "' message from client " << annotated_message.sender;
+
+  S::QueryInitiateResponseMessage proto_response;
+  CHECK(proto_response.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+}
+
+void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manager) {
+  const QueryHandle *query_handle = query_manager->query_handle();
+
+  const CatalogRelation *query_result = query_handle->getQueryResultRelation();
+  const tmb::client_id cli_id = query_handle->getClientId();
+  const std::size_t query_id = query_handle->query_id();
+
+  if (query_result == nullptr) {
+    // Clean up query execution states, i.e., QueryContext, in Shiftboss.
+    serialization::QueryTeardownMessage proto;
+    proto.set_query_id(query_id);
+
+    const size_t proto_length = proto.ByteSize();
+    char *proto_bytes = static_cast<char*>(malloc(proto_length));
+    CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+    TaggedMessage message(static_cast<const void*>(proto_bytes),
+                          proto_length,
+                          kQueryTeardownMessage);
+
+    // TODO(zuyu): Support multiple shiftbosses.
+    LOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
+              << "') to Shiftboss 0";
+    tmb::MessageBus::SendStatus send_status =
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           foreman_client_id_,
+                                           shiftboss_directory_->getClientId(0),
+                                           move(message));
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+        << " to Shiftboss";
+
+    TaggedMessage cli_message(kQueryExecutionSuccessMessage);
+
+    // Notify the CLI query execution successfully.
+    LOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '" << kQueryExecutionSuccessMessage
+              << "') to CLI with TMB client id " << cli_id;
+    send_status =
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           foreman_client_id_,
+                                           cli_id,
+                                           move(cli_message));
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+        << " to CLI with TMB client ID " << cli_id;
+    return;
+  }
+
+  // NOTE(zuyu): SaveQueryResultMessage implicitly triggers QueryTeardown in Shiftboss.
+  S::SaveQueryResultMessage proto;
+  proto.set_query_id(query_id);
+  proto.set_relation_id(query_result->getID());
+
+  const vector<block_id> blocks(query_result->getBlocksSnapshot());
+  for (const block_id block : blocks) {
+    proto.add_blocks(block);
+  }
+
+  proto.set_cli_id(cli_id);
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage message(static_cast<const void*>(proto_bytes),
+                        proto_length,
+                        kSaveQueryResultMessage);
+  free(proto_bytes);
+
+  LOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
+            << "') to Shiftboss 0";
+  // TODO(zuyu): Support multiple shiftbosses.
+  const tmb::MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         foreman_client_id_,
+                                         shiftboss_directory_->getClientId(0),
+                                         move(message));
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.hpp b/query_execution/PolicyEnforcerDistributed.hpp
new file mode 100644
index 0000000..16ebe36
--- /dev/null
+++ b/query_execution/PolicyEnforcerDistributed.hpp
@@ -0,0 +1,113 @@
+/**
+ * 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.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "query_execution/PolicyEnforcerBase.hpp"
+#include "query_execution/ShiftbossDirectory.hpp"
+#include "utility/Macros.hpp"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb {
+class MessageBus;
+class TaggedMessage;
+}
+
+namespace quickstep {
+
+class CatalogDatabaseLite;
+class QueryHandle;
+class QueryManagerBase;
+
+namespace serialization { class WorkOrderMessage; }
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief A class that ensures that a high level policy is maintained
+ *        in sharing resources among concurrent queries.
+ **/
+class PolicyEnforcerDistributed final : public PolicyEnforcerBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param foreman_client_id The TMB client ID of the Foreman.
+   * @param catalog_database The CatalogDatabase used.
+   * @param bus The TMB.
+   **/
+  PolicyEnforcerDistributed(const tmb::client_id foreman_client_id,
+                            CatalogDatabaseLite *catalog_database,
+                            ShiftbossDirectory *shiftboss_directory,
+                            tmb::MessageBus *bus,
+                            const bool profile_individual_workorders = false)
+      : PolicyEnforcerBase(catalog_database, profile_individual_workorders),
+        foreman_client_id_(foreman_client_id),
+        shiftboss_directory_(shiftboss_directory),
+        bus_(bus) {}
+
+  /**
+   * @brief Destructor.
+   **/
+  ~PolicyEnforcerDistributed() override {}
+
+  bool admitQuery(QueryHandle *query_handle) override;
+
+  /**
+   * @brief Get work order messages to be dispatched. These messages come from
+   *        the active queries.
+   *
+   * @param work_order_messages The work order messages to be dispatched.
+   **/
+  void getWorkOrderMessages(
+      std::vector<std::unique_ptr<serialization::WorkOrderMessage>> *work_order_messages);
+
+  /**
+   * @brief Process the initiate rebuild work order response message.
+   *
+   * @param tagged_message The message.
+   **/
+  void processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message);
+
+ private:
+  void decrementNumQueuedWorkOrders(const std::size_t shiftboss_index) override {
+    shiftboss_directory_->decrementNumQueuedWorkOrders(shiftboss_index);
+  }
+
+  void onQueryCompletion(QueryManagerBase *query_manager) override;
+
+  void initiateQueryInShiftboss(QueryHandle *query_handle);
+
+  const tmb::client_id foreman_client_id_;
+
+  ShiftbossDirectory *shiftboss_directory_;
+
+  tmb::MessageBus *bus_;
+
+  DISALLOW_COPY_AND_ASSIGN(PolicyEnforcerDistributed);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f680d35..20b684e 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -120,13 +120,25 @@ message InitiateRebuildResponseMessage {
   required uint64 shiftboss_index = 4;
 }
 
+message QueryTeardownMessage {
+  required uint64 query_id = 1;
+}
+
 message SaveQueryResultMessage {
-  required int32 relation_id = 1;
-  repeated fixed64 blocks = 2 [packed=true];
+  required uint64 query_id = 1;
+  required int32 relation_id = 2;
+  repeated fixed64 blocks = 3 [packed=true];
+
+  required uint32 cli_id = 4;  // tmb::client_id.
 }
 
 message SaveQueryResultResponseMessage {
   required int32 relation_id = 1;
+  required uint32 cli_id = 2;  // tmb::client_id.
+}
+
+message QueryExecutionSuccessMessage {
+  optional CatalogRelationSchema result_relation = 1;
 }
 
 // BlockLocator related messages.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 4643096..d154d84 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -86,9 +86,14 @@ enum QueryExecutionMessageType : message_type_id {
   kInitiateRebuildMessage,  // From Foreman to Shiftboss.
   kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
 
+  kQueryTeardownMessage,  // From Foreman to Shiftboss.
+
   kSaveQueryResultMessage,  // From Foreman to Shiftboss.
   kSaveQueryResultResponseMessage,  // From Shiftboss to Foreman.
 
+  // From Foreman to CLI.
+  kQueryExecutionSuccessMessage,
+
   // BlockLocator related messages, sorted in a life cycle of StorageManager
   // with a unique block domain.
   kBlockDomainRegistrationMessage,  // From Worker to BlockLocator.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index c60e323..8e37da8 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -37,7 +37,8 @@ using std::pair;
 namespace quickstep {
 
 QueryManagerBase::QueryManagerBase(QueryHandle *query_handle)
-    : query_id_(DCHECK_NOTNULL(query_handle)->query_id()),
+    : query_handle_(DCHECK_NOTNULL(query_handle)),
+      query_id_(query_handle->query_id()),
       query_dag_(DCHECK_NOTNULL(
           DCHECK_NOTNULL(query_handle->getQueryPlanMutable())->getQueryPlanDAGMutable())),
       num_operators_in_dag_(query_dag_->size()),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryManagerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.hpp b/query_execution/QueryManagerBase.hpp
index 782b8ed..a274742 100644
--- a/query_execution/QueryManagerBase.hpp
+++ b/query_execution/QueryManagerBase.hpp
@@ -74,6 +74,13 @@ class QueryManagerBase {
   virtual ~QueryManagerBase() {}
 
   /**
+   * @brief Get the query handle.
+   **/
+  const QueryHandle* query_handle() const {
+    return query_handle_;
+  }
+
+  /**
    * @brief Get the QueryExecutionState for this query.
    **/
   inline const QueryExecutionState& getQueryExecutionState() const {
@@ -252,9 +259,11 @@ class QueryManagerBase {
     return query_exec_state_->hasRebuildInitiated(index);
   }
 
+  const QueryHandle *query_handle_;
+
   const std::size_t query_id_;
 
-  DAG<RelationalOperator, bool> *query_dag_;
+  DAG<RelationalOperator, bool> *query_dag_;  // Owned by 'query_handle_'.
   const dag_node_index num_operators_in_dag_;
 
   // For all nodes, store their receiving dependents.


[39/50] [abbrv] incubator-quickstep git commit: Revert mode changes to Python scripts.

Posted by zu...@apache.org.
Revert mode changes to Python scripts.


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

Branch: refs/heads/travis-grpc
Commit: bd017482eb0e261d1ef64d7ff268932c3634cb90
Parents: 918167a
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Sun Aug 7 10:50:33 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Sun Aug 7 10:50:33 2016 -0700

----------------------------------------------------------------------
 cyclic_dependency.py   | 0
 validate_cmakelists.py | 0
 2 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bd017482/cyclic_dependency.py
----------------------------------------------------------------------
diff --git a/cyclic_dependency.py b/cyclic_dependency.py
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bd017482/validate_cmakelists.py
----------------------------------------------------------------------
diff --git a/validate_cmakelists.py b/validate_cmakelists.py
old mode 100644
new mode 100755


[11/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/CalculateInstalledMemory_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/CalculateInstalledMemory_unittest.cpp b/utility/tests/CalculateInstalledMemory_unittest.cpp
index 0bfec64..eb34f89 100644
--- a/utility/tests/CalculateInstalledMemory_unittest.cpp
+++ b/utility/tests/CalculateInstalledMemory_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/DAG_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/DAG_unittest.cpp b/utility/tests/DAG_unittest.cpp
index 30a8250..3fe2990 100644
--- a/utility/tests/DAG_unittest.cpp
+++ b/utility/tests/DAG_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/EqualsAnyConstant_benchmark.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/EqualsAnyConstant_benchmark.cpp b/utility/tests/EqualsAnyConstant_benchmark.cpp
index 75eeed9..a50a87f 100644
--- a/utility/tests/EqualsAnyConstant_benchmark.cpp
+++ b/utility/tests/EqualsAnyConstant_benchmark.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/EqualsAnyConstant_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/EqualsAnyConstant_unittest.cpp b/utility/tests/EqualsAnyConstant_unittest.cpp
index 5377180..c15fabb 100644
--- a/utility/tests/EqualsAnyConstant_unittest.cpp
+++ b/utility/tests/EqualsAnyConstant_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <array>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/HashPair_benchmark.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/HashPair_benchmark.cpp b/utility/tests/HashPair_benchmark.cpp
index 99a76a9..b3adadc 100644
--- a/utility/tests/HashPair_benchmark.cpp
+++ b/utility/tests/HashPair_benchmark.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <functional>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/PrimeNumber_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/PrimeNumber_unittest.cpp b/utility/tests/PrimeNumber_unittest.cpp
index 6b8e87a..60a55d5 100644
--- a/utility/tests/PrimeNumber_unittest.cpp
+++ b/utility/tests/PrimeNumber_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/ScopedDeleter_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/ScopedDeleter_unittest.cpp b/utility/tests/ScopedDeleter_unittest.cpp
index 558a37b..281f18c 100644
--- a/utility/tests/ScopedDeleter_unittest.cpp
+++ b/utility/tests/ScopedDeleter_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "gtest/gtest.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/SqlError_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/SqlError_unittest.cpp b/utility/tests/SqlError_unittest.cpp
index 7ed1072..416fe4b 100644
--- a/utility/tests/SqlError_unittest.cpp
+++ b/utility/tests/SqlError_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/SqlError.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/TemplateUtil_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/TemplateUtil_unittest.cpp b/utility/tests/TemplateUtil_unittest.cpp
index 7fcd318..ce5d662 100644
--- a/utility/tests/TemplateUtil_unittest.cpp
+++ b/utility/tests/TemplateUtil_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/TemplateUtil.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/TextBasedTestDriver_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/TextBasedTestDriver_unittest.cpp b/utility/tests/TextBasedTestDriver_unittest.cpp
index aca7e26..efe29e7 100644
--- a/utility/tests/TextBasedTestDriver_unittest.cpp
+++ b/utility/tests/TextBasedTestDriver_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/textbased_test/TextBasedTestDriver.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/ThreadSafeQueue_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/ThreadSafeQueue_unittest.cpp b/utility/tests/ThreadSafeQueue_unittest.cpp
index 0c96fdc..62ed817 100644
--- a/utility/tests/ThreadSafeQueue_unittest.cpp
+++ b/utility/tests/ThreadSafeQueue_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 // TODO(chasseur): This test is FAR from complete. For now it just does the

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/tests/TreeStringSerializable_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/TreeStringSerializable_unittest.cpp b/utility/tests/TreeStringSerializable_unittest.cpp
index 52c9df7..40108af 100644
--- a/utility/tests/TreeStringSerializable_unittest.cpp
+++ b/utility/tests/TreeStringSerializable_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/TreeStringSerializable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/textbased_test/TextBasedTest.cpp
----------------------------------------------------------------------
diff --git a/utility/textbased_test/TextBasedTest.cpp b/utility/textbased_test/TextBasedTest.cpp
index de4a87a..9a02f61 100644
--- a/utility/textbased_test/TextBasedTest.cpp
+++ b/utility/textbased_test/TextBasedTest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/textbased_test/TextBasedTest.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/textbased_test/TextBasedTest.hpp
----------------------------------------------------------------------
diff --git a/utility/textbased_test/TextBasedTest.hpp b/utility/textbased_test/TextBasedTest.hpp
index 7e38667..f0f35b6 100644
--- a/utility/textbased_test/TextBasedTest.hpp
+++ b/utility/textbased_test/TextBasedTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_TEXTBASED_TEST_TEXTBASED_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/textbased_test/TextBasedTestDriver.cpp
----------------------------------------------------------------------
diff --git a/utility/textbased_test/TextBasedTestDriver.cpp b/utility/textbased_test/TextBasedTestDriver.cpp
index a0d0942..5acfc8e 100644
--- a/utility/textbased_test/TextBasedTestDriver.cpp
+++ b/utility/textbased_test/TextBasedTestDriver.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "utility/textbased_test/TextBasedTestDriver.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/textbased_test/TextBasedTestDriver.hpp
----------------------------------------------------------------------
diff --git a/utility/textbased_test/TextBasedTestDriver.hpp b/utility/textbased_test/TextBasedTestDriver.hpp
index fc96a4b..95344a5 100644
--- a/utility/textbased_test/TextBasedTestDriver.hpp
+++ b/utility/textbased_test/TextBasedTestDriver.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_TEXTBASED_TEST_TEXTBASED_TEST_DRIVER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/utility/textbased_test/TextBasedTestRunner.hpp
----------------------------------------------------------------------
diff --git a/utility/textbased_test/TextBasedTestRunner.hpp b/utility/textbased_test/TextBasedTestRunner.hpp
index f088117..7a78967 100644
--- a/utility/textbased_test/TextBasedTestRunner.hpp
+++ b/utility/textbased_test/TextBasedTestRunner.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_UTILITY_TEXTBASED_TEST_TEXTBASED_TEST_RUNNER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/validate_cmakelists.py
----------------------------------------------------------------------
diff --git a/validate_cmakelists.py b/validate_cmakelists.py
old mode 100755
new mode 100644
index cf25d28..f691d1f
--- a/validate_cmakelists.py
+++ b/validate_cmakelists.py
@@ -16,20 +16,22 @@ TODO List / Known Issues & Limitations:
       detect missing third party libraries.
 """
 
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 from __future__ import absolute_import
 from __future__ import division

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/yarn/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/yarn/CMakeLists.txt b/yarn/CMakeLists.txt
index 5551973..dc36934 100644
--- a/yarn/CMakeLists.txt
+++ b/yarn/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 option(ENABLE_YARN "Enable optional support for Quickstep on YARN" OFF)
 



[19/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_reader_posix.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_reader_posix.h b/third_party/tmb/include/tmb/internal/log_reader_posix.h
index 5a4e74e..2dba5f3 100644
--- a/third_party/tmb/include/tmb/internal/log_reader_posix.h
+++ b/third_party/tmb/include/tmb/internal/log_reader_posix.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_READER_POSIX_H_
 #define TMB_INTERNAL_LOG_READER_POSIX_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_reader_stdio.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_reader_stdio.h b/third_party/tmb/include/tmb/internal/log_reader_stdio.h
index 9754e5b..6cb1d95 100644
--- a/third_party/tmb/include/tmb/internal/log_reader_stdio.h
+++ b/third_party/tmb/include/tmb/internal/log_reader_stdio.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_READER_STDIO_H_
 #define TMB_INTERNAL_LOG_READER_STDIO_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_record_header.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_record_header.h b/third_party/tmb/include/tmb/internal/log_record_header.h
index b57dcf0..a9fd47e 100644
--- a/third_party/tmb/include/tmb/internal/log_record_header.h
+++ b/third_party/tmb/include/tmb/internal/log_record_header.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_RECORD_HEADER_H_
 #define TMB_INTERNAL_LOG_RECORD_HEADER_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_writer_base.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_writer_base.h b/third_party/tmb/include/tmb/internal/log_writer_base.h
index 29679fe..020784c 100644
--- a/third_party/tmb/include/tmb/internal/log_writer_base.h
+++ b/third_party/tmb/include/tmb/internal/log_writer_base.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_WRITER_BASE_H_
 #define TMB_INTERNAL_LOG_WRITER_BASE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_writer_posix.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_writer_posix.h b/third_party/tmb/include/tmb/internal/log_writer_posix.h
index 3c209f3..4d51e09 100644
--- a/third_party/tmb/include/tmb/internal/log_writer_posix.h
+++ b/third_party/tmb/include/tmb/internal/log_writer_posix.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_WRITER_POSIX_H_
 #define TMB_INTERNAL_LOG_WRITER_POSIX_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_writer_stdio.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_writer_stdio.h b/third_party/tmb/include/tmb/internal/log_writer_stdio.h
index fdcfa46..1823a94 100644
--- a/third_party/tmb/include/tmb/internal/log_writer_stdio.h
+++ b/third_party/tmb/include/tmb/internal/log_writer_stdio.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_WRITER_STDIO_H_
 #define TMB_INTERNAL_LOG_WRITER_STDIO_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/logging_constants.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/logging_constants.h b/third_party/tmb/include/tmb/internal/logging_constants.h
index 967ac52..f4dd6b9 100644
--- a/third_party/tmb/include/tmb/internal/logging_constants.h
+++ b/third_party/tmb/include/tmb/internal/logging_constants.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOGGING_CONSTANTS_H_
 #define TMB_INTERNAL_LOGGING_CONSTANTS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/memory_based_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/memory_based_message_bus.h b/third_party/tmb/include/tmb/internal/memory_based_message_bus.h
index a0f0e5a..2e31bd4 100644
--- a/third_party/tmb/include/tmb/internal/memory_based_message_bus.h
+++ b/third_party/tmb/include/tmb/internal/memory_based_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_MEMORY_BASED_MESSAGE_BUS_H_
 #define TMB_INTERNAL_MEMORY_BASED_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/memory_mirror_cancellation_set.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/memory_mirror_cancellation_set.h b/third_party/tmb/include/tmb/internal/memory_mirror_cancellation_set.h
index 729c160..c4e9c7b 100644
--- a/third_party/tmb/include/tmb/internal/memory_mirror_cancellation_set.h
+++ b/third_party/tmb/include/tmb/internal/memory_mirror_cancellation_set.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_MEMORY_MIRROR_CANCELLATION_SET_H_
 #define TMB_INTERNAL_MEMORY_MIRROR_CANCELLATION_SET_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/memory_mirror_delete_batch.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/memory_mirror_delete_batch.h b/third_party/tmb/include/tmb/internal/memory_mirror_delete_batch.h
index adbae5f..8227858 100644
--- a/third_party/tmb/include/tmb/internal/memory_mirror_delete_batch.h
+++ b/third_party/tmb/include/tmb/internal/memory_mirror_delete_batch.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_MEMORY_MIRROR_DELETE_BATCH_H_
 #define TMB_INTERNAL_MEMORY_MIRROR_DELETE_BATCH_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/message_metadata.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/message_metadata.h b/third_party/tmb/include/tmb/internal/message_metadata.h
index f9380ae..9b29543 100644
--- a/third_party/tmb/include/tmb/internal/message_metadata.h
+++ b/third_party/tmb/include/tmb/internal/message_metadata.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_MESSAGE_METADATA_H_
 #define TMB_INTERNAL_MESSAGE_METADATA_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/native_transaction_log.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/native_transaction_log.h b/third_party/tmb/include/tmb/internal/native_transaction_log.h
index 5d21615..9a4ee27 100644
--- a/third_party/tmb/include/tmb/internal/native_transaction_log.h
+++ b/third_party/tmb/include/tmb/internal/native_transaction_log.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_NATIVE_TRANSACTION_LOG_H_
 #define TMB_INTERNAL_NATIVE_TRANSACTION_LOG_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/net_memory_container_pusher.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/net_memory_container_pusher.h b/third_party/tmb/include/tmb/internal/net_memory_container_pusher.h
index b796532..ca7f242 100644
--- a/third_party/tmb/include/tmb/internal/net_memory_container_pusher.h
+++ b/third_party/tmb/include/tmb/internal/net_memory_container_pusher.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_NET_MEMORY_CONTAINER_PUSHER_H_
 #define TMB_INTERNAL_NET_MEMORY_CONTAINER_PUSHER_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/net_message_removal_interface.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/net_message_removal_interface.h b/third_party/tmb/include/tmb/internal/net_message_removal_interface.h
index 678d211..80ffd70 100644
--- a/third_party/tmb/include/tmb/internal/net_message_removal_interface.h
+++ b/third_party/tmb/include/tmb/internal/net_message_removal_interface.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_NET_MESSAGE_REMOVAL_INTERFACE_H_
 #define TMB_INTERNAL_NET_MESSAGE_REMOVAL_INTERFACE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/net_service_impl.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/net_service_impl.h b/third_party/tmb/include/tmb/internal/net_service_impl.h
index b0ab81a..b9289ce 100644
--- a/third_party/tmb/include/tmb/internal/net_service_impl.h
+++ b/third_party/tmb/include/tmb/internal/net_service_impl.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_NET_SERVICE_IMPL_H_
 #define TMB_INTERNAL_NET_SERVICE_IMPL_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/persistent_bus_state_interface.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/persistent_bus_state_interface.h b/third_party/tmb/include/tmb/internal/persistent_bus_state_interface.h
index 9fe1779..5b88195 100644
--- a/third_party/tmb/include/tmb/internal/persistent_bus_state_interface.h
+++ b/third_party/tmb/include/tmb/internal/persistent_bus_state_interface.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_PERSISTENT_BUS_STATE_INTERFACE_H_
 #define TMB_INTERNAL_PERSISTENT_BUS_STATE_INTERFACE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/queued_message.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/queued_message.h b/third_party/tmb/include/tmb/internal/queued_message.h
index ba86bfb..d954bd6 100644
--- a/third_party/tmb/include/tmb/internal/queued_message.h
+++ b/third_party/tmb/include/tmb/internal/queued_message.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_QUEUED_MESSAGE_H_
 #define TMB_INTERNAL_QUEUED_MESSAGE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/rcu.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/rcu.h b/third_party/tmb/include/tmb/internal/rcu.h
index 2007d32..76e6ff4 100644
--- a/third_party/tmb/include/tmb/internal/rcu.h
+++ b/third_party/tmb/include/tmb/internal/rcu.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_RCU_H_
 #define TMB_INTERNAL_RCU_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/shared_bool.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/shared_bool.h b/third_party/tmb/include/tmb/internal/shared_bool.h
index a63239a..8e09ca6 100644
--- a/third_party/tmb/include/tmb/internal/shared_bool.h
+++ b/third_party/tmb/include/tmb/internal/shared_bool.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_SHARED_BOOL_H_
 #define TMB_INTERNAL_SHARED_BOOL_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/sqlite_connection.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/sqlite_connection.h b/third_party/tmb/include/tmb/internal/sqlite_connection.h
index f4ecaaf..c928c1d 100644
--- a/third_party/tmb/include/tmb/internal/sqlite_connection.h
+++ b/third_party/tmb/include/tmb/internal/sqlite_connection.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_SQLITE_CONNECTION_H_
 #define TMB_INTERNAL_SQLITE_CONNECTION_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/sqlite_connection_pool.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/sqlite_connection_pool.h b/third_party/tmb/include/tmb/internal/sqlite_connection_pool.h
index b997235..33cd497 100644
--- a/third_party/tmb/include/tmb/internal/sqlite_connection_pool.h
+++ b/third_party/tmb/include/tmb/internal/sqlite_connection_pool.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_SQLITE_CONNECTION_POOL_H_
 #define TMB_INTERNAL_SQLITE_CONNECTION_POOL_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/threadsafe_set.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/threadsafe_set.h b/third_party/tmb/include/tmb/internal/threadsafe_set.h
index a699b76..ccdafdd 100644
--- a/third_party/tmb/include/tmb/internal/threadsafe_set.h
+++ b/third_party/tmb/include/tmb/internal/threadsafe_set.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_THREADSAFE_SET_H_
 #define TMB_INTERNAL_THREADSAFE_SET_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/tree_receiver_message_queue.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/tree_receiver_message_queue.h b/third_party/tmb/include/tmb/internal/tree_receiver_message_queue.h
index 2e44d17..72860f3 100644
--- a/third_party/tmb/include/tmb/internal/tree_receiver_message_queue.h
+++ b/third_party/tmb/include/tmb/internal/tree_receiver_message_queue.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_TREE_RECEIVER_MESSAGE_QUEUE_H_
 #define TMB_INTERNAL_TREE_RECEIVER_MESSAGE_QUEUE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/voltdb_connection_pool.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/voltdb_connection_pool.h b/third_party/tmb/include/tmb/internal/voltdb_connection_pool.h
index bdf1218..4bfa258 100644
--- a/third_party/tmb/include/tmb/internal/voltdb_connection_pool.h
+++ b/third_party/tmb/include/tmb/internal/voltdb_connection_pool.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_VOLTDB_CONNECTION_POOL_H_
 #define TMB_INTERNAL_VOLTDB_CONNECTION_POOL_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/voltdb_procedure_factory.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/voltdb_procedure_factory.h b/third_party/tmb/include/tmb/internal/voltdb_procedure_factory.h
index 48c6f0c..d801d81 100644
--- a/third_party/tmb/include/tmb/internal/voltdb_procedure_factory.h
+++ b/third_party/tmb/include/tmb/internal/voltdb_procedure_factory.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_VOLTDB_PROCEDURE_FACTORY_H_
 #define TMB_INTERNAL_VOLTDB_PROCEDURE_FACTORY_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/voltdb_procedure_warehouse.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/voltdb_procedure_warehouse.h b/third_party/tmb/include/tmb/internal/voltdb_procedure_warehouse.h
index 30ba001..2804770 100644
--- a/third_party/tmb/include/tmb/internal/voltdb_procedure_warehouse.h
+++ b/third_party/tmb/include/tmb/internal/voltdb_procedure_warehouse.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_VOLTDB_PROCEDURE_WAREHOUSE_H_
 #define TMB_INTERNAL_VOLTDB_PROCEDURE_WAREHOUSE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/zookeeper_format.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/zookeeper_format.h b/third_party/tmb/include/tmb/internal/zookeeper_format.h
index e83b9bd..f061a53 100644
--- a/third_party/tmb/include/tmb/internal/zookeeper_format.h
+++ b/third_party/tmb/include/tmb/internal/zookeeper_format.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_ZOOKEEPER_FORMAT_H_
 #define TMB_INTERNAL_ZOOKEEPER_FORMAT_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/zookeeper_receiver_context.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/zookeeper_receiver_context.h b/third_party/tmb/include/tmb/internal/zookeeper_receiver_context.h
index f785e2a..b2a4297 100644
--- a/third_party/tmb/include/tmb/internal/zookeeper_receiver_context.h
+++ b/third_party/tmb/include/tmb/internal/zookeeper_receiver_context.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_ZOOKEEPER_RECEIVER_CONTEXT_H_
 #define TMB_INTERNAL_ZOOKEEPER_RECEIVER_CONTEXT_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/zookeeper_transaction_batch.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/zookeeper_transaction_batch.h b/third_party/tmb/include/tmb/internal/zookeeper_transaction_batch.h
index 6183798..a1b7f41 100644
--- a/third_party/tmb/include/tmb/internal/zookeeper_transaction_batch.h
+++ b/third_party/tmb/include/tmb/internal/zookeeper_transaction_batch.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_ZOOKEEPER_TRANSACTION_BATCH_H_
 #define TMB_INTERNAL_ZOOKEEPER_TRANSACTION_BATCH_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/leveldb_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/leveldb_message_bus.h b/third_party/tmb/include/tmb/leveldb_message_bus.h
index d9b05ff..6000b92 100644
--- a/third_party/tmb/include/tmb/leveldb_message_bus.h
+++ b/third_party/tmb/include/tmb/leveldb_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_LEVELDB_MESSAGE_BUS_H_
 #define TMB_LEVELDB_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/memory_mirror_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/memory_mirror_message_bus.h b/third_party/tmb/include/tmb/memory_mirror_message_bus.h
index 660a3ba..fedb970 100644
--- a/third_party/tmb/include/tmb/memory_mirror_message_bus.h
+++ b/third_party/tmb/include/tmb/memory_mirror_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_MEMORY_MIRROR_MESSAGE_BUS_H_
 #define TMB_MEMORY_MIRROR_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/message_bus.h b/third_party/tmb/include/tmb/message_bus.h
index 7d8ff4c..a4ca525 100644
--- a/third_party/tmb/include/tmb/message_bus.h
+++ b/third_party/tmb/include/tmb/message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_MESSAGE_BUS_H_
 #define TMB_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/message_style.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/message_style.h b/third_party/tmb/include/tmb/message_style.h
index 3a3cfdd..e4bdab0 100644
--- a/third_party/tmb/include/tmb/message_style.h
+++ b/third_party/tmb/include/tmb/message_style.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_MESSAGE_STYLE_H_
 #define TMB_MESSAGE_STYLE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/native_logging_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/native_logging_message_bus.h b/third_party/tmb/include/tmb/native_logging_message_bus.h
index 1cd9634..e905125 100644
--- a/third_party/tmb/include/tmb/native_logging_message_bus.h
+++ b/third_party/tmb/include/tmb/native_logging_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_NATIVE_LOGGING_MESSAGE_BUS_H_
 #define TMB_NATIVE_LOGGING_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/native_net_client_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/native_net_client_message_bus.h b/third_party/tmb/include/tmb/native_net_client_message_bus.h
index d960855..c0181d3 100644
--- a/third_party/tmb/include/tmb/native_net_client_message_bus.h
+++ b/third_party/tmb/include/tmb/native_net_client_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_NATIVE_NET_CLIENT_MESSAGE_BUS_H_
 #define TMB_NATIVE_NET_CLIENT_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/priority.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/priority.h b/third_party/tmb/include/tmb/priority.h
index 173b833..d4b6584 100644
--- a/third_party/tmb/include/tmb/priority.h
+++ b/third_party/tmb/include/tmb/priority.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_PRIORITY_H_
 #define TMB_PRIORITY_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/pure_memory_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/pure_memory_message_bus.h b/third_party/tmb/include/tmb/pure_memory_message_bus.h
index 57990dd..f62b557 100644
--- a/third_party/tmb/include/tmb/pure_memory_message_bus.h
+++ b/third_party/tmb/include/tmb/pure_memory_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_PURE_MEMORY_MESSAGE_BUS_H_
 #define TMB_PURE_MEMORY_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/sqlite_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/sqlite_message_bus.h b/third_party/tmb/include/tmb/sqlite_message_bus.h
index 79dc776..17f28b3 100644
--- a/third_party/tmb/include/tmb/sqlite_message_bus.h
+++ b/third_party/tmb/include/tmb/sqlite_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_SQLITE_MESSAGE_BUS_H_
 #define TMB_SQLITE_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/tagged_message.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/tagged_message.h b/third_party/tmb/include/tmb/tagged_message.h
index f3a77ee..93f49da 100644
--- a/third_party/tmb/include/tmb/tagged_message.h
+++ b/third_party/tmb/include/tmb/tagged_message.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_TAGGED_MESSAGE_H_
 #define TMB_TAGGED_MESSAGE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/voltdb_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/voltdb_message_bus.h b/third_party/tmb/include/tmb/voltdb_message_bus.h
index f58fed8..2c83f3e 100644
--- a/third_party/tmb/include/tmb/voltdb_message_bus.h
+++ b/third_party/tmb/include/tmb/voltdb_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_VOLTDB_MESSAGE_BUS_H_
 #define TMB_VOLTDB_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/zookeeper_message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/zookeeper_message_bus.h b/third_party/tmb/include/tmb/zookeeper_message_bus.h
index 41d9ff6..75468dc 100644
--- a/third_party/tmb/include/tmb/zookeeper_message_bus.h
+++ b/third_party/tmb/include/tmb/zookeeper_message_bus.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_ZOOKEEPER_MESSAGE_BUS_H_
 #define TMB_ZOOKEEPER_MESSAGE_BUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/crc32.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/crc32.cc b/third_party/tmb/src/crc32.cc
index c35b4e0..53e1a2b 100644
--- a/third_party/tmb/src/crc32.cc
+++ b/third_party/tmb/src/crc32.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 <cstddef>
 #include <cstdint>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/heap_receiver_message_queue.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/heap_receiver_message_queue.cc b/third_party/tmb/src/heap_receiver_message_queue.cc
index 7cba753..2ab5fa7 100644
--- a/third_party/tmb/src/heap_receiver_message_queue.cc
+++ b/third_party/tmb/src/heap_receiver_message_queue.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/heap_receiver_message_queue.h"
 



[44/50] [abbrv] incubator-quickstep git commit: Fixed bugs in creating WorkOrderProtos.

Posted by zu...@apache.org.
Fixed bugs in creating WorkOrderProtos.


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

Branch: refs/heads/travis-grpc
Commit: 2c0ce6a3bcf2ec40b0c32d077552cda3e225f787
Parents: 85e02de
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 18:42:32 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 10:42:25 2016 -0700

----------------------------------------------------------------------
 relational_operators/HashJoinOperator.cpp | 1 +
 relational_operators/WorkOrderFactory.cpp | 4 +---
 2 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2c0ce6a3/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 7851f41..779c0fe 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -393,6 +393,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *
 serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block) {
   serialization::WorkOrder *proto = new serialization::WorkOrder;
   proto->set_work_order_type(serialization::HASH_JOIN);
+  proto->set_query_id(query_id_);
 
   proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type,
                       serialization::HashJoinWorkOrder::HASH_OUTER_JOIN);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2c0ce6a3/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 721d735..7d7af59 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -533,13 +533,11 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
         return false;
       }
 
-      const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
       const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
       for (int i = 0; i < proto.ExtensionSize(serialization::HashJoinWorkOrder::join_key_attributes); ++i) {
         const attribute_id attr_id =
             proto.GetExtension(serialization::HashJoinWorkOrder::join_key_attributes, i);
-        if (!build_relation.hasAttributeWithId(attr_id) ||
-            !probe_relation.hasAttributeWithId(attr_id)) {
+        if (!probe_relation.hasAttributeWithId(attr_id)) {
           return false;
         }
       }


[49/50] [abbrv] incubator-quickstep git commit: Disabled shared build.

Posted by zu...@apache.org.
Disabled shared build.


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

Branch: refs/heads/travis-grpc
Commit: cc392553c1e8a917ec96c16a457fb7fd2951a526
Parents: 9ec7c4b
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jun 14 20:04:51 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Fri Aug 12 09:04:15 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/cc392553/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 3c00cd7..2650b6c 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -56,7 +56,6 @@ before_script:
   - $PROTOC --version
   - (cd build &&
      cmake -D CMAKE_BUILD_TYPE=$BUILD_TYPE
-           -D BUILD_SHARED_LIBS=On
            -D CMAKE_C_FLAGS_DEBUG="$DEBUG_FLAGS"
            -D CMAKE_CXX_FLAGS_DEBUG="$DEBUG_FLAGS"
            -D CMAKE_C_FLAGS_RELEASE="$RELEASE_FLAGS"


[08/50] [abbrv] incubator-quickstep git commit: Processed InitiateRebuild message for the distributed version.

Posted by zu...@apache.org.
Processed InitiateRebuild message for the distributed version.


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

Branch: refs/heads/travis-grpc
Commit: 991f7a4eb3923a9fb43f88e1fe2b5ea057ee92e8
Parents: 5e5ec9e
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Wed Aug 3 15:28:00 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Thu Aug 4 09:45:57 2016 -0700

----------------------------------------------------------------------
 query_execution/QueryExecutionMessages.proto |  1 +
 query_execution/QueryManagerDistributed.cpp  | 23 +++++++++++++++++++++++
 query_execution/QueryManagerDistributed.hpp  | 14 +++++++++++++-
 query_execution/Shiftboss.cpp                |  2 ++
 4 files changed, 39 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/991f7a4e/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 5a089d2..4922042 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -113,6 +113,7 @@ message InitiateRebuildResponseMessage {
   required uint64 query_id = 1;
   required uint64 operator_index = 2;
   required uint64 num_rebuild_work_orders = 3;
+  required uint64 shiftboss_index = 4;
 }
 
 message SaveQueryResultMessage {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/991f7a4e/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index e906fa5..5d589d5 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -119,6 +119,28 @@ bool QueryManagerDistributed::fetchNormalWorkOrders(const dag_node_index index)
   return generated_new_workorder_protos;
 }
 
+void QueryManagerDistributed::processInitiateRebuildResponseMessage(const dag_node_index op_index,
+                                                                    const std::size_t num_rebuild_work_orders) {
+  // TODO(zuyu): Multiple Shiftbosses support.
+  query_exec_state_->setRebuildStatus(op_index, num_rebuild_work_orders, true);
+
+  if (num_rebuild_work_orders != 0u) {
+    // Wait for the rebuild work orders to finish.
+    return;
+  }
+
+  // No needs for rebuilds.
+  markOperatorFinished(op_index);
+
+  for (const std::pair<dag_node_index, bool> &dependent_link :
+       query_dag_->getDependents(op_index)) {
+    const dag_node_index dependent_op_index = dependent_link.first;
+    if (checkAllBlockingDependenciesMet(dependent_op_index)) {
+      processOperator(dependent_op_index, true);
+    }
+  }
+}
+
 bool QueryManagerDistributed::initiateRebuild(const dag_node_index index) {
   DCHECK(checkRebuildRequired(index));
   DCHECK(!checkRebuildInitiated(index));
@@ -127,6 +149,7 @@ bool QueryManagerDistributed::initiateRebuild(const dag_node_index index) {
   DCHECK_NE(op.getInsertDestinationID(), QueryContext::kInvalidInsertDestinationId);
 
   serialization::InitiateRebuildMessage proto;
+  proto.set_query_id(query_id_);
   proto.set_operator_index(index);
   proto.set_insert_destination_index(op.getInsertDestinationID());
   proto.set_relation_id(op.getOutputRelationID());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/991f7a4e/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 8641c22..5d9be81 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -15,6 +15,7 @@
 #ifndef QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_DISTRIBUTED_HPP_
 #define QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_DISTRIBUTED_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "query_execution/QueryExecutionState.hpp"
@@ -60,7 +61,18 @@ class QueryManagerDistributed final : public QueryManagerBase {
 
   bool fetchNormalWorkOrders(const dag_node_index index) override;
 
- /**
+  /**
+   * @brief Process the initiate rebuild work order response message.
+   *
+   * @param op_index The index of the specified operator node in the query DAG
+   *        for initiating the rebuild work order.
+   * @param num_rebuild_work_orders The number of the rebuild work orders
+   *        generated for the operator indexed by 'op_index'.
+   **/
+  void processInitiateRebuildResponseMessage(const dag_node_index op_index,
+                                             const std::size_t num_rebuild_work_orders);
+
+  /**
    * @brief Get the next normal workorder to be excuted, wrapped in a
    *        WorkOrderMessage proto.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/991f7a4e/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 7f655c6..3f7129b 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -311,6 +311,8 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   proto.set_query_id(query_id);
   proto.set_operator_index(op_index);
   proto.set_num_rebuild_work_orders(partially_filled_block_refs.size());
+  // TODO(zuyu): Multiple Shiftboss support.
+  proto.set_shiftboss_index(0);
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));


[18/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/CancelMessages.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/CancelMessages.java b/third_party/tmb/src/java/CancelMessages.java
index 0465f43..86a1d6a 100644
--- a/third_party/tmb/src/java/CancelMessages.java
+++ b/third_party/tmb/src/java/CancelMessages.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/ConnectClient.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/ConnectClient.java b/third_party/tmb/src/java/ConnectClient.java
index 5980895..34f443e 100644
--- a/third_party/tmb/src/java/ConnectClient.java
+++ b/third_party/tmb/src/java/ConnectClient.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/DeleteMessages.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/DeleteMessages.java b/third_party/tmb/src/java/DeleteMessages.java
index a7507db..9fe0a65 100644
--- a/third_party/tmb/src/java/DeleteMessages.java
+++ b/third_party/tmb/src/java/DeleteMessages.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/DeleteMessagesUnchecked.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/DeleteMessagesUnchecked.java b/third_party/tmb/src/java/DeleteMessagesUnchecked.java
index f1d6b3e..90a047f 100644
--- a/third_party/tmb/src/java/DeleteMessagesUnchecked.java
+++ b/third_party/tmb/src/java/DeleteMessagesUnchecked.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/DisconnectClient.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/DisconnectClient.java b/third_party/tmb/src/java/DisconnectClient.java
index 5bc6805..ea0361a 100644
--- a/third_party/tmb/src/java/DisconnectClient.java
+++ b/third_party/tmb/src/java/DisconnectClient.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/LoadState.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/LoadState.java b/third_party/tmb/src/java/LoadState.java
index 78bd499..60c6b12 100644
--- a/third_party/tmb/src/java/LoadState.java
+++ b/third_party/tmb/src/java/LoadState.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/Receive.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/Receive.java b/third_party/tmb/src/java/Receive.java
index 2e131a2..a929ee9 100644
--- a/third_party/tmb/src/java/Receive.java
+++ b/third_party/tmb/src/java/Receive.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/ReceiveAndDelete.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/ReceiveAndDelete.java b/third_party/tmb/src/java/ReceiveAndDelete.java
index debf8bb..dd10836 100644
--- a/third_party/tmb/src/java/ReceiveAndDelete.java
+++ b/third_party/tmb/src/java/ReceiveAndDelete.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/RegisterReceiver.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/RegisterReceiver.java b/third_party/tmb/src/java/RegisterReceiver.java
index 2176fba..a164257 100644
--- a/third_party/tmb/src/java/RegisterReceiver.java
+++ b/third_party/tmb/src/java/RegisterReceiver.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/RegisterSender.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/RegisterSender.java b/third_party/tmb/src/java/RegisterSender.java
index c5d8216..54dc64d 100644
--- a/third_party/tmb/src/java/RegisterSender.java
+++ b/third_party/tmb/src/java/RegisterSender.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/ResetBus.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/ResetBus.java b/third_party/tmb/src/java/ResetBus.java
index c2ae5aa..3050c99 100644
--- a/third_party/tmb/src/java/ResetBus.java
+++ b/third_party/tmb/src/java/ResetBus.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/SendToAny.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/SendToAny.java b/third_party/tmb/src/java/SendToAny.java
index a293f0e..240b550 100644
--- a/third_party/tmb/src/java/SendToAny.java
+++ b/third_party/tmb/src/java/SendToAny.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/SendToExplicitReceivers.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/SendToExplicitReceivers.java b/third_party/tmb/src/java/SendToExplicitReceivers.java
index 62d3cfa..11acf8d 100644
--- a/third_party/tmb/src/java/SendToExplicitReceivers.java
+++ b/third_party/tmb/src/java/SendToExplicitReceivers.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/SendToExplicitReceiversUnchecked.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/SendToExplicitReceiversUnchecked.java b/third_party/tmb/src/java/SendToExplicitReceiversUnchecked.java
index 9b04fed..4108834 100644
--- a/third_party/tmb/src/java/SendToExplicitReceiversUnchecked.java
+++ b/third_party/tmb/src/java/SendToExplicitReceiversUnchecked.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/SendToSingleExplicitReceiver.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/SendToSingleExplicitReceiver.java b/third_party/tmb/src/java/SendToSingleExplicitReceiver.java
index 36ad02c..983132c 100644
--- a/third_party/tmb/src/java/SendToSingleExplicitReceiver.java
+++ b/third_party/tmb/src/java/SendToSingleExplicitReceiver.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/java/SendToSingleExplicitReceiverUnchecked.java
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/java/SendToSingleExplicitReceiverUnchecked.java b/third_party/tmb/src/java/SendToSingleExplicitReceiverUnchecked.java
index ba3541a..0e2e650 100644
--- a/third_party/tmb/src/java/SendToSingleExplicitReceiverUnchecked.java
+++ b/third_party/tmb/src/java/SendToSingleExplicitReceiverUnchecked.java
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 package tmb.voltdb;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/leveldb_key_comparator.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/leveldb_key_comparator.cc b/third_party/tmb/src/leveldb_key_comparator.cc
index 6055aea..0c04893 100644
--- a/third_party/tmb/src/leveldb_key_comparator.cc
+++ b/third_party/tmb/src/leveldb_key_comparator.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/leveldb_key_comparator.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/leveldb_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/leveldb_message_bus.cc b/third_party/tmb/src/leveldb_message_bus.cc
index 925d42b..43a3d51 100644
--- a/third_party/tmb/src/leveldb_message_bus.cc
+++ b/third_party/tmb/src/leveldb_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/leveldb_message_bus.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/log_reader_posix.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/log_reader_posix.cc b/third_party/tmb/src/log_reader_posix.cc
index 5e9ffa4..91a9ed1 100644
--- a/third_party/tmb/src/log_reader_posix.cc
+++ b/third_party/tmb/src/log_reader_posix.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/log_reader_posix.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/log_reader_stdio.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/log_reader_stdio.cc b/third_party/tmb/src/log_reader_stdio.cc
index 3b4aeb8..4ecfa26 100644
--- a/third_party/tmb/src/log_reader_stdio.cc
+++ b/third_party/tmb/src/log_reader_stdio.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/log_reader_stdio.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/log_writer_posix.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/log_writer_posix.cc b/third_party/tmb/src/log_writer_posix.cc
index 6ccdb86..d121d17 100644
--- a/third_party/tmb/src/log_writer_posix.cc
+++ b/third_party/tmb/src/log_writer_posix.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/log_writer_posix.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/log_writer_stdio.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/log_writer_stdio.cc b/third_party/tmb/src/log_writer_stdio.cc
index 19dd5dc..752b4b0 100644
--- a/third_party/tmb/src/log_writer_stdio.cc
+++ b/third_party/tmb/src/log_writer_stdio.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/log_writer_stdio.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/memory_based_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/memory_based_message_bus.cc b/third_party/tmb/src/memory_based_message_bus.cc
index d4cd9c9..b6de408 100644
--- a/third_party/tmb/src/memory_based_message_bus.cc
+++ b/third_party/tmb/src/memory_based_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/memory_based_message_bus.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/memory_mirror_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/memory_mirror_message_bus.cc b/third_party/tmb/src/memory_mirror_message_bus.cc
index f4d0cbc..7420358 100644
--- a/third_party/tmb/src/memory_mirror_message_bus.cc
+++ b/third_party/tmb/src/memory_mirror_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/memory_mirror_message_bus.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/message_bus.cc b/third_party/tmb/src/message_bus.cc
index a213971..44324ec 100644
--- a/third_party/tmb/src/message_bus.cc
+++ b/third_party/tmb/src/message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/message_bus.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/native_net_client_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/native_net_client_message_bus.cc b/third_party/tmb/src/native_net_client_message_bus.cc
index 31c76a9..a1e6f73 100644
--- a/third_party/tmb/src/native_net_client_message_bus.cc
+++ b/third_party/tmb/src/native_net_client_message_bus.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 // TODO(chasseur): Better error handling in place of asserts.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/native_transaction_log.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/native_transaction_log.cc b/third_party/tmb/src/native_transaction_log.cc
index 84f1202..1a0f614 100644
--- a/third_party/tmb/src/native_transaction_log.cc
+++ b/third_party/tmb/src/native_transaction_log.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/native_transaction_log.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/net_service_impl.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/net_service_impl.cc b/third_party/tmb/src/net_service_impl.cc
index 579f7ae..6c640f4 100644
--- a/third_party/tmb/src/net_service_impl.cc
+++ b/third_party/tmb/src/net_service_impl.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 "tmb/internal/net_service_impl.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/proto/tmb_net.proto
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/proto/tmb_net.proto b/third_party/tmb/src/proto/tmb_net.proto
index f020199..4c03c17 100644
--- a/third_party/tmb/src/proto/tmb_net.proto
+++ b/third_party/tmb/src/proto/tmb_net.proto
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 syntax = "proto3";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/pure_memory_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/pure_memory_message_bus.cc b/third_party/tmb/src/pure_memory_message_bus.cc
index d6a8799..f89cee0 100644
--- a/third_party/tmb/src/pure_memory_message_bus.cc
+++ b/third_party/tmb/src/pure_memory_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/pure_memory_message_bus.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/sql/voltdb_schema.sql
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/sql/voltdb_schema.sql b/third_party/tmb/src/sql/voltdb_schema.sql
index c1df2e5..06b4783 100644
--- a/third_party/tmb/src/sql/voltdb_schema.sql
+++ b/third_party/tmb/src/sql/voltdb_schema.sql
@@ -1,16 +1,19 @@
---   Copyright 2014-2015 Quickstep Technologies LLC.
+-- 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
 --
---   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
 --
---       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.
+-- 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.
 
 CREATE TABLE client(
     client_id INTEGER NOT NULL,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/sqlite_connection.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/sqlite_connection.cc b/third_party/tmb/src/sqlite_connection.cc
index 60e38ae..ba2e1e3 100644
--- a/third_party/tmb/src/sqlite_connection.cc
+++ b/third_party/tmb/src/sqlite_connection.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/sqlite_connection.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/sqlite_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/sqlite_message_bus.cc b/third_party/tmb/src/sqlite_message_bus.cc
index d1653e9..34bee57 100644
--- a/third_party/tmb/src/sqlite_message_bus.cc
+++ b/third_party/tmb/src/sqlite_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 // TODO(chasseur): Better error handling in place of asserts.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/tmb_net_server.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/tmb_net_server.cc b/third_party/tmb/src/tmb_net_server.cc
index 9a0b157..c2be7ef 100644
--- a/third_party/tmb/src/tmb_net_server.cc
+++ b/third_party/tmb/src/tmb_net_server.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 <cassert>
 #include <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/tree_receiver_message_queue.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/tree_receiver_message_queue.cc b/third_party/tmb/src/tree_receiver_message_queue.cc
index 4dda2c3..7844d8f 100644
--- a/third_party/tmb/src/tree_receiver_message_queue.cc
+++ b/third_party/tmb/src/tree_receiver_message_queue.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/tree_receiver_message_queue.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/voltdb_connection_pool.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/voltdb_connection_pool.cc b/third_party/tmb/src/voltdb_connection_pool.cc
index 1acf505..e2b0bfb 100644
--- a/third_party/tmb/src/voltdb_connection_pool.cc
+++ b/third_party/tmb/src/voltdb_connection_pool.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/voltdb_connection_pool.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/voltdb_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/voltdb_message_bus.cc b/third_party/tmb/src/voltdb_message_bus.cc
index c337921..130a5ea 100644
--- a/third_party/tmb/src/voltdb_message_bus.cc
+++ b/third_party/tmb/src/voltdb_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 // TODO(chasseur): Better error handling in place of asserts (also handle
 // exceptions thrown by VoltDB).

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/voltdb_procedure_factory.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/voltdb_procedure_factory.cc b/third_party/tmb/src/voltdb_procedure_factory.cc
index 6953de0..75cc88e 100644
--- a/third_party/tmb/src/voltdb_procedure_factory.cc
+++ b/third_party/tmb/src/voltdb_procedure_factory.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/voltdb_procedure_factory.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/zookeeper_message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/zookeeper_message_bus.cc b/third_party/tmb/src/zookeeper_message_bus.cc
index 73013ca..e78d51e 100644
--- a/third_party/tmb/src/zookeeper_message_bus.cc
+++ b/third_party/tmb/src/zookeeper_message_bus.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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.
+ **/
 
 // TODO(chasseur): Better error handling in place of asserts.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/src/zookeeper_transaction_batch.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/src/zookeeper_transaction_batch.cc b/third_party/tmb/src/zookeeper_transaction_batch.cc
index ddeb610..b5adf86 100644
--- a/third_party/tmb/src/zookeeper_transaction_batch.cc
+++ b/third_party/tmb/src/zookeeper_transaction_batch.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "tmb/internal/zookeeper_transaction_batch.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/leveldb_message_bus_async_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/leveldb_message_bus_async_unittest.cc b/third_party/tmb/tests/leveldb_message_bus_async_unittest.cc
index 694c800..f1332e2 100644
--- a/third_party/tmb/tests/leveldb_message_bus_async_unittest.cc
+++ b/third_party/tmb/tests/leveldb_message_bus_async_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/leveldb_message_bus_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/leveldb_message_bus_unittest.cc b/third_party/tmb/tests/leveldb_message_bus_unittest.cc
index 201006f..900fabf 100644
--- a/third_party/tmb/tests/leveldb_message_bus_unittest.cc
+++ b/third_party/tmb/tests/leveldb_message_bus_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/memory_mirror_message_bus_with_leveldb_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/memory_mirror_message_bus_with_leveldb_unittest.cc b/third_party/tmb/tests/memory_mirror_message_bus_with_leveldb_unittest.cc
index 1e2286e..f01cec9 100644
--- a/third_party/tmb/tests/memory_mirror_message_bus_with_leveldb_unittest.cc
+++ b/third_party/tmb/tests/memory_mirror_message_bus_with_leveldb_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/memory_mirror_message_bus_with_sqlite_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/memory_mirror_message_bus_with_sqlite_unittest.cc b/third_party/tmb/tests/memory_mirror_message_bus_with_sqlite_unittest.cc
index cfa24d5..8f1a7d5 100644
--- a/third_party/tmb/tests/memory_mirror_message_bus_with_sqlite_unittest.cc
+++ b/third_party/tmb/tests/memory_mirror_message_bus_with_sqlite_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/tests/memory_mirror_message_bus_with_voltdb_unittest.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/tests/memory_mirror_message_bus_with_voltdb_unittest.cc b/third_party/tmb/tests/memory_mirror_message_bus_with_voltdb_unittest.cc
index cd4e3b5..9d86449 100644
--- a/third_party/tmb/tests/memory_mirror_message_bus_with_voltdb_unittest.cc
+++ b/third_party/tmb/tests/memory_mirror_message_bus_with_voltdb_unittest.cc
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 "gtest/gtest.h"
 #include "tests/message_bus_unittest_common.h"



[30/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/PhysicalGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.hpp b/query_optimizer/PhysicalGenerator.hpp
index 484900a..886a173 100644
--- a/query_optimizer/PhysicalGenerator.hpp
+++ b/query_optimizer/PhysicalGenerator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_GENERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/QueryHandle.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryHandle.hpp b/query_optimizer/QueryHandle.hpp
index 55427cd..1ca6021 100644
--- a/query_optimizer/QueryHandle.hpp
+++ b/query_optimizer/QueryHandle.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_QUERY_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/QueryOptimizerConfig.h.in
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryOptimizerConfig.h.in b/query_optimizer/QueryOptimizerConfig.h.in
index da0fa18..3bdb6ce 100644
--- a/query_optimizer/QueryOptimizerConfig.h.in
+++ b/query_optimizer/QueryOptimizerConfig.h.in
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_DISTRIBUTED

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/QueryOptimizerModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryOptimizerModule.hpp b/query_optimizer/QueryOptimizerModule.hpp
index 8e9eed3..d7cb00a 100644
--- a/query_optimizer/QueryOptimizerModule.hpp
+++ b/query_optimizer/QueryOptimizerModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup QueryOptimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/QueryPlan.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryPlan.hpp b/query_optimizer/QueryPlan.hpp
index 2560e71..5cd174c 100644
--- a/query_optimizer/QueryPlan.hpp
+++ b/query_optimizer/QueryPlan.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_QUERY_PLAN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/QueryProcessor.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.cpp b/query_optimizer/QueryProcessor.cpp
index 059e5f1..ea9f126 100644
--- a/query_optimizer/QueryProcessor.cpp
+++ b/query_optimizer/QueryProcessor.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/QueryProcessor.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/QueryProcessor.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.hpp b/query_optimizer/QueryProcessor.hpp
index 23a0189..2a7e942 100644
--- a/query_optimizer/QueryProcessor.hpp
+++ b/query_optimizer/QueryProcessor.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_QUERY_PROCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/Validator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Validator.hpp b/query_optimizer/Validator.hpp
index 394efcd..9f8b6fd 100644
--- a/query_optimizer/Validator.hpp
+++ b/query_optimizer/Validator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_VALIDATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 5d5b596..abbc3da 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_queryoptimizer_costmodel_CostModel ../../empty_src.cpp CostModel.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/CostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CostModel.hpp b/query_optimizer/cost_model/CostModel.hpp
index 2420404..ca49733 100644
--- a/query_optimizer/cost_model/CostModel.hpp
+++ b/query_optimizer/cost_model/CostModel.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUERY_OPTIMIZER_COST_MODEL_COST_MODEL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/CostModelModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CostModelModule.hpp b/query_optimizer/cost_model/CostModelModule.hpp
index bbf03ae..41bc053 100644
--- a/query_optimizer/cost_model/CostModelModule.hpp
+++ b/query_optimizer/cost_model/CostModelModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup CostModel

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index e5222ff..f313c90 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/cost_model/SimpleCostModel.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/SimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.hpp b/query_optimizer/cost_model/SimpleCostModel.hpp
index 9837039..25ff8fe 100644
--- a/query_optimizer/cost_model/SimpleCostModel.hpp
+++ b/query_optimizer/cost_model/SimpleCostModel.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUERY_OPTIMIZER_COST_MODEL_SIMPLE_COST_MODEL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index badfeb1..911a765 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 83032cf..4314b92 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUERY_OPTIMIZER_COST_MODEL_STAR_SCHEMA_SIMPLE_COST_MODEL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/AggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/AggregateFunction.cpp b/query_optimizer/expressions/AggregateFunction.cpp
index e061f03..cf2cb00 100644
--- a/query_optimizer/expressions/AggregateFunction.cpp
+++ b/query_optimizer/expressions/AggregateFunction.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/AggregateFunction.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/AggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/AggregateFunction.hpp b/query_optimizer/expressions/AggregateFunction.hpp
index 86698fd..108c661 100644
--- a/query_optimizer/expressions/AggregateFunction.hpp
+++ b/query_optimizer/expressions/AggregateFunction.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_AGGREGATE_FUNCTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Alias.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Alias.cpp b/query_optimizer/expressions/Alias.cpp
index c529957..30dade7 100644
--- a/query_optimizer/expressions/Alias.cpp
+++ b/query_optimizer/expressions/Alias.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/Alias.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Alias.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Alias.hpp b/query_optimizer/expressions/Alias.hpp
index 0400207..e07f9fe 100644
--- a/query_optimizer/expressions/Alias.hpp
+++ b/query_optimizer/expressions/Alias.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_ALIAS_HPP

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/AttributeReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/AttributeReference.cpp b/query_optimizer/expressions/AttributeReference.cpp
index b15d482..f0e49d4 100644
--- a/query_optimizer/expressions/AttributeReference.cpp
+++ b/query_optimizer/expressions/AttributeReference.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/AttributeReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/AttributeReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/AttributeReference.hpp b/query_optimizer/expressions/AttributeReference.hpp
index f299bbc..f5207b1 100644
--- a/query_optimizer/expressions/AttributeReference.hpp
+++ b/query_optimizer/expressions/AttributeReference.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_ATTRIBUTE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/BinaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/BinaryExpression.cpp b/query_optimizer/expressions/BinaryExpression.cpp
index fcef90a..446dd55 100644
--- a/query_optimizer/expressions/BinaryExpression.cpp
+++ b/query_optimizer/expressions/BinaryExpression.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/BinaryExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/BinaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/BinaryExpression.hpp b/query_optimizer/expressions/BinaryExpression.hpp
index 69cbd11..9b11ed1 100644
--- a/query_optimizer/expressions/BinaryExpression.hpp
+++ b/query_optimizer/expressions/BinaryExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_BINARY_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index d12644a..35fac90 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_queryoptimizer_expressions_AggregateFunction AggregateFunction.cpp AggregateFunction.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Cast.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.cpp b/query_optimizer/expressions/Cast.cpp
index f55a7c2..c0813c5 100644
--- a/query_optimizer/expressions/Cast.cpp
+++ b/query_optimizer/expressions/Cast.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/Cast.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Cast.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.hpp b/query_optimizer/expressions/Cast.hpp
index c4bd360..ac5bd02 100644
--- a/query_optimizer/expressions/Cast.hpp
+++ b/query_optimizer/expressions/Cast.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_CAST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ComparisonExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ComparisonExpression.cpp b/query_optimizer/expressions/ComparisonExpression.cpp
index 5cebf8b..8d93794 100644
--- a/query_optimizer/expressions/ComparisonExpression.cpp
+++ b/query_optimizer/expressions/ComparisonExpression.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/ComparisonExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ComparisonExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ComparisonExpression.hpp b/query_optimizer/expressions/ComparisonExpression.hpp
index 8a84e75..011e04e 100644
--- a/query_optimizer/expressions/ComparisonExpression.hpp
+++ b/query_optimizer/expressions/ComparisonExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_COMPARISON_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Exists.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Exists.cpp b/query_optimizer/expressions/Exists.cpp
index 2bb5a6b..a7b0201 100644
--- a/query_optimizer/expressions/Exists.cpp
+++ b/query_optimizer/expressions/Exists.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/Exists.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Exists.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Exists.hpp b/query_optimizer/expressions/Exists.hpp
index 6293020..0b75de5 100644
--- a/query_optimizer/expressions/Exists.hpp
+++ b/query_optimizer/expressions/Exists.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_EXISTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ExprId.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExprId.hpp b/query_optimizer/expressions/ExprId.hpp
index bc9cb4b..be5dd6a 100644
--- a/query_optimizer/expressions/ExprId.hpp
+++ b/query_optimizer/expressions/ExprId.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER__EXPRESSIONS_EXPR_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Expression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Expression.hpp b/query_optimizer/expressions/Expression.hpp
index 82e3dc5..7127047 100644
--- a/query_optimizer/expressions/Expression.hpp
+++ b/query_optimizer/expressions/Expression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ExpressionType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionType.hpp b/query_optimizer/expressions/ExpressionType.hpp
index 77e0874..5008f1d 100644
--- a/query_optimizer/expressions/ExpressionType.hpp
+++ b/query_optimizer/expressions/ExpressionType.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_EXPRESSION_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ExpressionUtil.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.cpp b/query_optimizer/expressions/ExpressionUtil.cpp
index 3ccd3c9..c9541ef 100644
--- a/query_optimizer/expressions/ExpressionUtil.cpp
+++ b/query_optimizer/expressions/ExpressionUtil.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/ExpressionUtil.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index 4c35719..e9a4067 100644
--- a/query_optimizer/expressions/ExpressionUtil.hpp
+++ b/query_optimizer/expressions/ExpressionUtil.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_EXPRESSION_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/InTableQuery.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/InTableQuery.cpp b/query_optimizer/expressions/InTableQuery.cpp
index b4abbe0..7122664 100644
--- a/query_optimizer/expressions/InTableQuery.cpp
+++ b/query_optimizer/expressions/InTableQuery.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/InTableQuery.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/InTableQuery.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/InTableQuery.hpp b/query_optimizer/expressions/InTableQuery.hpp
index e4abf22..8e6d0e7 100644
--- a/query_optimizer/expressions/InTableQuery.hpp
+++ b/query_optimizer/expressions/InTableQuery.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_IN_TABLE_QUERY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/InValueList.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/InValueList.cpp b/query_optimizer/expressions/InValueList.cpp
index dd77d95..0451db7 100644
--- a/query_optimizer/expressions/InValueList.cpp
+++ b/query_optimizer/expressions/InValueList.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/InValueList.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/InValueList.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/InValueList.hpp b/query_optimizer/expressions/InValueList.hpp
index 9fc2ace..0ef3bba 100644
--- a/query_optimizer/expressions/InValueList.hpp
+++ b/query_optimizer/expressions/InValueList.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_IN_VALUE_LIST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/LogicalAnd.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/LogicalAnd.cpp b/query_optimizer/expressions/LogicalAnd.cpp
index 3e47cfc..80ee0eb 100644
--- a/query_optimizer/expressions/LogicalAnd.cpp
+++ b/query_optimizer/expressions/LogicalAnd.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/LogicalAnd.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/LogicalAnd.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/LogicalAnd.hpp b/query_optimizer/expressions/LogicalAnd.hpp
index 360c6b4..8152728 100644
--- a/query_optimizer/expressions/LogicalAnd.hpp
+++ b/query_optimizer/expressions/LogicalAnd.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_LOGICAL_AND_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/LogicalNot.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/LogicalNot.cpp b/query_optimizer/expressions/LogicalNot.cpp
index b3ae662..dce13a9 100644
--- a/query_optimizer/expressions/LogicalNot.cpp
+++ b/query_optimizer/expressions/LogicalNot.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/LogicalNot.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/LogicalNot.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/LogicalNot.hpp b/query_optimizer/expressions/LogicalNot.hpp
index 8c009e0..90a98cd 100644
--- a/query_optimizer/expressions/LogicalNot.hpp
+++ b/query_optimizer/expressions/LogicalNot.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_LOGICAL_NOT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/LogicalOr.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/LogicalOr.cpp b/query_optimizer/expressions/LogicalOr.cpp
index c738951..02028e1 100644
--- a/query_optimizer/expressions/LogicalOr.cpp
+++ b/query_optimizer/expressions/LogicalOr.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/LogicalOr.hpp"



[37/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index bf9ed8d..af17264 100644
--- a/README.md
+++ b/README.md
@@ -8,7 +8,7 @@
 ## What is Quickstep?
 Apache Quickstep is high-performance database engine designed to exploit the full potential of hardware that is packed in modern computing boxes (servers and laptops). The initial version (available now!) targets single-node in-memory environments. If your data spills overs the memory limit Quickstep will still work, so you don't have to obsessively worry about the in-memory part. Also, if your working set fits in memory then Quickstep will transparently and automatically figure that out, and cache that hot set to  deliver in-memory performance.
 
-Distributed execution is the next big feature for Quickstep.  
+Distributed execution is the next big feature for Quickstep.
 
 Quickstep began life in 2011 as a
 [research project at the University of Wisconsin](https://www.cs.wisc.edu/~jignesh)
@@ -39,13 +39,13 @@ And, it is **open source!**
 3. Initialize the dependencies: ```git submodule init```
 4. Checkout the dependencies: ```git submodule update```
 5. Go into the build directory: ```cd build```
-6. Create the Makefile: ```cmake -D CMAKE_BUILD_TYPE=Release ..```  
-7. Build: ```make -j4```. Note you may replace the 4 with the number of cores 
+6. Create the Makefile: ```cmake -D CMAKE_BUILD_TYPE=Release ..```
+7. Build: ```make -j4```. Note you may replace the 4 with the number of cores
    on your machine.
-8. Start quickstep: ```./quickstep_cli_shell --initialize_db=true```. You can 
-   now fire SQL queries. To quit, you can type in ```quit;``` Your data is 
+8. Start quickstep: ```./quickstep_cli_shell --initialize_db=true```. You can
+   now fire SQL queries. To quit, you can type in ```quit;``` Your data is
    stored in the directory ```qsstor```. Note the next time you start Quickstep,
-   you can omit the ``` --initialize_db``` flag (as the database has already 
+   you can omit the ``` --initialize_db``` flag (as the database has already
    been initialized), and simply start Quickstep as: ```./quickstep_cli_shell```.
    There are also a number of optional flags that you can specify, and to see
    the full list, you can type in: ```./quickstep_cli_shell --help```
@@ -87,21 +87,21 @@ CREATE TABLE City (cid Integer, name VARCHAR(80), state CHAR(2));
   SELECT cid, MIN(lowTemperature), MAX(highTemperature) FROM Weather GROUP BY cid;
   ```
 
-  c. Find the min and max temperature for each city using a nested query, and 
+  c. Find the min and max temperature for each city using a nested query, and
      printing thie city name:
   ```
   SELECT * FROM City C, (SELECT cid, MIN(lowTemperature), MAX(highTemperature) FROM Weather GROUP BY cid) AS T WHERE C.cid = T.cid;
   ```
 
 12. Quickstep also supports a COPY TABLE command. If you want to try that, then
-    from a separate shell file type in the following: 
+    from a separate shell file type in the following:
 
     ```
     echo "3|2015-11-3|49|29" > /tmp/tmp.tbl
     echo "3|2015-11-4|48|28" >> /tmp/tmp.tbl
     echo "3|2015-11-5|47|27" >> /tmp/tmp.tbl
     ```
-   
+
     Then, load this new data by typing the following SQL in the Quickstep shell:
 
     ```

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/WORKING_WITH_AN_IDE.md
----------------------------------------------------------------------
diff --git a/WORKING_WITH_AN_IDE.md b/WORKING_WITH_AN_IDE.md
index f9139ab..017a174 100644
--- a/WORKING_WITH_AN_IDE.md
+++ b/WORKING_WITH_AN_IDE.md
@@ -37,7 +37,7 @@ ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/
 ```
 
 Once you have brew installed, simply use brew to install cmake from the
-Terminal app by typing: 
+Terminal app by typing:
 
 ```
 brew install cmake
@@ -90,13 +90,13 @@ cp -R ../qsstor .
 ```
 
 The above command ensures that you now have the appropriate directory structure
-and a starting catalog file to start Quickstep. The default database is called 
-simply "default" and no relations in it upfront. 
+and a starting catalog file to start Quickstep. The default database is called
+simply "default" and no relations in it upfront.
 
 There are other ways of specifying where Quickstep stores the data and catalog.
 In particular there is a  `-storage_path` option that could be an alternative
 way of specifying the storage path and avoiding the copy above. You can find
-these and other command line options by typing: 
+these and other command line options by typing:
 
 ```
 ./quickstep_cli_shell -help
@@ -104,15 +104,15 @@ these and other command line options by typing:
 
 
 ###4: Debug Quickstep
-Now you can debug as you would any normal process in Xcode. Note the 
+Now you can debug as you would any normal process in Xcode. Note the
 linenoise option in the cmake command above is important if you are going
-to run quickstep from Xcode (by hitting the "play" button). If you are 
-curious why we have that option, see 
+to run quickstep from Xcode (by hitting the "play" button). If you are
+curious why we have that option, see
 https://github.com/antirez/linenoise/issues/85. Quickstep uses the linenoise
 package, and Xcode's embedded terminal has limited functionality. With the
 cmake option above, we turn off using linenoise.
 
-Sometimes you may want to run quickstep from the command line and still 
+Sometimes you may want to run quickstep from the command line and still
 debug with Xcode. For that scenario, you do the following:
 a) In Xcode, go to `Debug -> Attach to Processs` and type in
 `quickstep_cli_shell`. If you want to set a breakpoint, you can do
@@ -136,7 +136,7 @@ you can examine the stack in Xcode.
 
 ###5: Unit Tests
 Individual unit tests show up as target schemas, so you can simply select them
-and run the unit test of interest. 
+and run the unit test of interest.
 
 Running all the unit tests is complicated, and simply picking the `RUN_TESTS`
 does not work. So, this is a known limitation at this point. You can, however,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/build/profile_build.sh
----------------------------------------------------------------------
diff --git a/build/profile_build.sh b/build/profile_build.sh
index b7164ad..7d56265 100755
--- a/build/profile_build.sh
+++ b/build/profile_build.sh
@@ -17,17 +17,22 @@
 # is not supported.
 # If CMakeCache.txt is detected, the script skips cmake and runs make only.
 
-#   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
+# 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
+#   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.
+# 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.
 
 set -e
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/build/vagrant/debian-jessie-amd64/Vagrantfile
----------------------------------------------------------------------
diff --git a/build/vagrant/debian-jessie-amd64/Vagrantfile b/build/vagrant/debian-jessie-amd64/Vagrantfile
index 95712f6..255fbff 100644
--- a/build/vagrant/debian-jessie-amd64/Vagrantfile
+++ b/build/vagrant/debian-jessie-amd64/Vagrantfile
@@ -1,6 +1,19 @@
-# This file copyright (c) 2015, Pivotal Software, Inc.
-# All rights reserved.
-# See file CREDITS.txt for details.
+# 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.
 
 # Basic Vagrant config (API version 2)
 Vagrant.configure(2) do |config|

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/build/vagrant/freebsd-10.2-amd64/Vagrantfile
----------------------------------------------------------------------
diff --git a/build/vagrant/freebsd-10.2-amd64/Vagrantfile b/build/vagrant/freebsd-10.2-amd64/Vagrantfile
index 5863af6..cce52ab 100644
--- a/build/vagrant/freebsd-10.2-amd64/Vagrantfile
+++ b/build/vagrant/freebsd-10.2-amd64/Vagrantfile
@@ -1,6 +1,19 @@
-# This file copyright (c) 2015, Pivotal Software, Inc.
-# All rights reserved.
-# See file CREDITS.txt for details.
+# 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.
 
 # Basic Vagrant config (API version 2)
 Vagrant.configure(2) do |config|

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/build/vagrant/ubuntu-precise-amd64/Vagrantfile
----------------------------------------------------------------------
diff --git a/build/vagrant/ubuntu-precise-amd64/Vagrantfile b/build/vagrant/ubuntu-precise-amd64/Vagrantfile
index 903dedc..863c9db 100644
--- a/build/vagrant/ubuntu-precise-amd64/Vagrantfile
+++ b/build/vagrant/ubuntu-precise-amd64/Vagrantfile
@@ -1,6 +1,19 @@
-# This file copyright (c) 2015, Pivotal Software, Inc.
-# All rights reserved.
-# See file CREDITS.txt for details.
+# 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.
 
 # Basic Vagrant config (API version 2)
 Vagrant.configure(2) do |config|

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index 64b4f16..dd4ef99 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 if(LIBNUMA_FOUND)
   set(QUICKSTEP_HAVE_LIBNUMA TRUE)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/Catalog.cpp
----------------------------------------------------------------------
diff --git a/catalog/Catalog.cpp b/catalog/Catalog.cpp
index 0ccb160..369d529 100644
--- a/catalog/Catalog.cpp
+++ b/catalog/Catalog.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/Catalog.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/Catalog.hpp
----------------------------------------------------------------------
diff --git a/catalog/Catalog.hpp b/catalog/Catalog.hpp
index ee663d2..daaec8c 100644
--- a/catalog/Catalog.hpp
+++ b/catalog/Catalog.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index ce4bc2e..90ce37e 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//     University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 
@@ -82,7 +82,7 @@ message IndexScheme {
 
 message CatalogRelationStatistics {
   optional fixed64 num_tuples = 1;
-  
+
   message NumDistinctValuesEntry {
     required int32 attr_id = 1;
     required fixed64 num_distinct_values = 2;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogAttribute.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogAttribute.cpp b/catalog/CatalogAttribute.cpp
index 488e30c..e9630d6 100644
--- a/catalog/CatalogAttribute.cpp
+++ b/catalog/CatalogAttribute.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/CatalogAttribute.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogAttribute.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogAttribute.hpp b/catalog/CatalogAttribute.hpp
index 253aad9..7f7fafb 100644
--- a/catalog/CatalogAttribute.hpp
+++ b/catalog/CatalogAttribute.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_ATTRIBUTE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogConfig.h.in
----------------------------------------------------------------------
diff --git a/catalog/CatalogConfig.h.in b/catalog/CatalogConfig.h.in
index ec28f58..dd68577 100644
--- a/catalog/CatalogConfig.h.in
+++ b/catalog/CatalogConfig.h.in
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_HAVE_LIBNUMA

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogDatabase.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabase.cpp b/catalog/CatalogDatabase.cpp
index 5d45b6e..c95196c 100644
--- a/catalog/CatalogDatabase.cpp
+++ b/catalog/CatalogDatabase.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/CatalogDatabase.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogDatabase.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabase.hpp b/catalog/CatalogDatabase.hpp
index 0f784f4..a0804a2 100644
--- a/catalog/CatalogDatabase.hpp
+++ b/catalog/CatalogDatabase.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_DATABASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogDatabaseCache.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabaseCache.cpp b/catalog/CatalogDatabaseCache.cpp
index ec8f47a..7f7c495 100644
--- a/catalog/CatalogDatabaseCache.cpp
+++ b/catalog/CatalogDatabaseCache.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/CatalogDatabaseCache.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogDatabaseCache.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabaseCache.hpp b/catalog/CatalogDatabaseCache.hpp
index b3e73a6..88ca61d 100644
--- a/catalog/CatalogDatabaseCache.hpp
+++ b/catalog/CatalogDatabaseCache.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_DATABASE_CACHE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogDatabaseLite.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabaseLite.hpp b/catalog/CatalogDatabaseLite.hpp
index e2f8389..0c846a0 100644
--- a/catalog/CatalogDatabaseLite.hpp
+++ b/catalog/CatalogDatabaseLite.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_DATABASE_LITE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogErrors.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogErrors.hpp b/catalog/CatalogErrors.hpp
index cc5529d..b156daa 100644
--- a/catalog/CatalogErrors.hpp
+++ b/catalog/CatalogErrors.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_ERRORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogModule.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogModule.hpp b/catalog/CatalogModule.hpp
index 357d6d5..e1ae851 100644
--- a/catalog/CatalogModule.hpp
+++ b/catalog/CatalogModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Catalog

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogRelation.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.cpp b/catalog/CatalogRelation.cpp
index 682b6be..c288532 100644
--- a/catalog/CatalogRelation.cpp
+++ b/catalog/CatalogRelation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/CatalogRelation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogRelation.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.hpp b/catalog/CatalogRelation.hpp
index e0d5350..c38e526 100644
--- a/catalog/CatalogRelation.hpp
+++ b/catalog/CatalogRelation.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_RELATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogRelationSchema.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.cpp b/catalog/CatalogRelationSchema.cpp
index 97c834f..5568cef 100644
--- a/catalog/CatalogRelationSchema.cpp
+++ b/catalog/CatalogRelationSchema.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/CatalogRelationSchema.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogRelationSchema.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.hpp b/catalog/CatalogRelationSchema.hpp
index d773bc7..d8b6128 100644
--- a/catalog/CatalogRelationSchema.hpp
+++ b/catalog/CatalogRelationSchema.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_RELATION_SCHEMA_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 2bd92b4..6a51570 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/CatalogRelationStatistics.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index 572d141..f2056f3 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_RELATION_STATISTICS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/CatalogTypedefs.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogTypedefs.hpp b/catalog/CatalogTypedefs.hpp
index 44832b6..f7a2d53 100644
--- a/catalog/CatalogTypedefs.hpp
+++ b/catalog/CatalogTypedefs.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_CATALOG_TYPEDEFS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/IndexScheme.cpp
----------------------------------------------------------------------
diff --git a/catalog/IndexScheme.cpp b/catalog/IndexScheme.cpp
index 6ad3706..c686c39 100644
--- a/catalog/IndexScheme.cpp
+++ b/catalog/IndexScheme.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/IndexScheme.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/IndexScheme.hpp
----------------------------------------------------------------------
diff --git a/catalog/IndexScheme.hpp b/catalog/IndexScheme.hpp
index d4d67dc..3bf3f17 100644
--- a/catalog/IndexScheme.hpp
+++ b/catalog/IndexScheme.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_INDEX_SCHEME_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/NUMAPlacementScheme.cpp
----------------------------------------------------------------------
diff --git a/catalog/NUMAPlacementScheme.cpp b/catalog/NUMAPlacementScheme.cpp
index 46ae560..8e7d0fa 100644
--- a/catalog/NUMAPlacementScheme.cpp
+++ b/catalog/NUMAPlacementScheme.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/NUMAPlacementScheme.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/NUMAPlacementScheme.hpp
----------------------------------------------------------------------
diff --git a/catalog/NUMAPlacementScheme.hpp b/catalog/NUMAPlacementScheme.hpp
index 2f68a04..e5ab83a 100644
--- a/catalog/NUMAPlacementScheme.hpp
+++ b/catalog/NUMAPlacementScheme.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_PARTITION_PLACEMENT_SCHEME_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/PartitionScheme.cpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionScheme.cpp b/catalog/PartitionScheme.cpp
index 6229ce6..d6e3469 100644
--- a/catalog/PartitionScheme.cpp
+++ b/catalog/PartitionScheme.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/PartitionScheme.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/PartitionScheme.hpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionScheme.hpp b/catalog/PartitionScheme.hpp
index 0906115..c882a5e 100644
--- a/catalog/PartitionScheme.hpp
+++ b/catalog/PartitionScheme.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_PARTITION_SCHEME_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/PartitionSchemeHeader.cpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionSchemeHeader.cpp b/catalog/PartitionSchemeHeader.cpp
index 66f4c02..26c317d 100644
--- a/catalog/PartitionSchemeHeader.cpp
+++ b/catalog/PartitionSchemeHeader.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "catalog/PartitionSchemeHeader.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/PartitionSchemeHeader.hpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionSchemeHeader.hpp b/catalog/PartitionSchemeHeader.hpp
index 7b9fcce..89efc6d 100644
--- a/catalog/PartitionSchemeHeader.hpp
+++ b/catalog/PartitionSchemeHeader.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CATALOG_PARTITION_SCHEME_HEADER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/tests/Catalog_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/Catalog_unittest.cpp b/catalog/tests/Catalog_unittest.cpp
index 8ee55a6..e430b89 100644
--- a/catalog/tests/Catalog_unittest.cpp
+++ b/catalog/tests/Catalog_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/tests/NUMAPlacementScheme_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/NUMAPlacementScheme_unittest.cpp b/catalog/tests/NUMAPlacementScheme_unittest.cpp
index 7320b3b..6a3b32f 100644
--- a/catalog/tests/NUMAPlacementScheme_unittest.cpp
+++ b/catalog/tests/NUMAPlacementScheme_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <numa.h>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/catalog/tests/PartitionScheme_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/PartitionScheme_unittest.cpp b/catalog/tests/PartitionScheme_unittest.cpp
index 259d5ae..d10b26e 100644
--- a/catalog/tests/PartitionScheme_unittest.cpp
+++ b/catalog/tests/PartitionScheme_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index 9637055..b86821a 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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_directories(${CMAKE_CURRENT_BINARY_DIR})
 add_subdirectory(tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/CliConfig.h.in
----------------------------------------------------------------------
diff --git a/cli/CliConfig.h.in b/cli/CliConfig.h.in
index 69f4d8a..2508f3a 100644
--- a/cli/CliConfig.h.in
+++ b/cli/CliConfig.h.in
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_USE_LINENOISE

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/CliModule.hpp
----------------------------------------------------------------------
diff --git a/cli/CliModule.hpp b/cli/CliModule.hpp
index eca047d..e65de98 100644
--- a/cli/CliModule.hpp
+++ b/cli/CliModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup CLI

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 82c1ad9..78fbe6f 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/CommandExecutor.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/CommandExecutor.hpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.hpp b/cli/CommandExecutor.hpp
index 19d03e6..25b47cc 100644
--- a/cli/CommandExecutor.hpp
+++ b/cli/CommandExecutor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_COMMAND_COMMAND_EXECUTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/DefaultsConfigurator.hpp
----------------------------------------------------------------------
diff --git a/cli/DefaultsConfigurator.hpp b/cli/DefaultsConfigurator.hpp
index 4da05b2..7450d88 100644
--- a/cli/DefaultsConfigurator.hpp
+++ b/cli/DefaultsConfigurator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_CLI_DEFAULTS_CONFIGURATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/cli/DropRelation.cpp
----------------------------------------------------------------------
diff --git a/cli/DropRelation.cpp b/cli/DropRelation.cpp
index b34c84a..9bf75f2 100644
--- a/cli/DropRelation.cpp
+++ b/cli/DropRelation.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "cli/DropRelation.hpp"


[10/50] [abbrv] incubator-quickstep git commit: Refactored QueryProcessor.

Posted by zu...@apache.org.
Refactored QueryProcessor.


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

Branch: refs/heads/travis-grpc
Commit: 410d2b672d06ec2f4bed7a22fdd0075135157923
Parents: bf7646f
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Thu Aug 4 14:19:38 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Fri Aug 5 11:18:33 2016 -0700

----------------------------------------------------------------------
 query_optimizer/QueryProcessor.cpp | 8 +++++---
 query_optimizer/QueryProcessor.hpp | 8 +-------
 2 files changed, 6 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/410d2b67/query_optimizer/QueryProcessor.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.cpp b/query_optimizer/QueryProcessor.cpp
index f3693a1..059e5f1 100644
--- a/query_optimizer/QueryProcessor.cpp
+++ b/query_optimizer/QueryProcessor.cpp
@@ -32,9 +32,11 @@ namespace quickstep {
 
 void QueryProcessor::generateQueryHandle(const ParseStatement &statement,
                                          QueryHandle *query_handle) {
-  optimizer_->generateQueryHandle(statement, query_handle);
+  optimizer::Optimizer optimizer(getDefaultDatabase(), storage_manager_.get());
 
-  if (optimizer_->isCatalogChanged() && !catalog_altered_) {
+  optimizer.generateQueryHandle(statement, query_handle);
+
+  if (optimizer.isCatalogChanged() && !catalog_altered_) {
     catalog_altered_ = true;
   }
 
@@ -66,7 +68,7 @@ void QueryProcessor::loadCatalog() {
     throw CatalogNotProto(catalog_filename_);
   }
   catalog_file.close();
-  catalog_.reset(new Catalog(catalog_proto));
+  catalog_ = std::make_unique<Catalog>(catalog_proto);
 
   catalog_altered_ = false;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/410d2b67/query_optimizer/QueryProcessor.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.hpp b/query_optimizer/QueryProcessor.hpp
index 43e462f..23a0189 100644
--- a/query_optimizer/QueryProcessor.hpp
+++ b/query_optimizer/QueryProcessor.hpp
@@ -26,7 +26,6 @@
 #include <string>
 
 #include "catalog/Catalog.hpp"
-#include "query_optimizer/Optimizer.hpp"
 #include "storage/StorageManager.hpp"
 #include "utility/Macros.hpp"
 
@@ -136,13 +135,10 @@ class QueryProcessor {
   QueryProcessor(const std::string &catalog_filename,
                  const std::string &storage_path)
       : catalog_filename_(catalog_filename),
+        storage_manager_(std::make_unique<StorageManager>(storage_path)),
         catalog_altered_(false),
         query_id_(0) {
     loadCatalog();
-    storage_manager_.reset(new StorageManager(storage_path));
-
-    // Construct after Catalog loads and StorageManager initializes.
-    optimizer_.reset(new optimizer::Optimizer(getDefaultDatabase(), storage_manager_.get()));
   }
 
   /**
@@ -203,8 +199,6 @@ class QueryProcessor {
   std::unique_ptr<Catalog> catalog_;
   std::unique_ptr<StorageManager> storage_manager_;
 
-  std::unique_ptr<optimizer::Optimizer> optimizer_;
-
   bool catalog_altered_;
 
   std::size_t query_id_;


[38/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
QUICKSTEP-40: Fix Copyright notice to confirm to Apache.


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

Branch: refs/heads/travis-grpc
Commit: 918167a444e0ec0b479713afe1bd20b404685b21
Parents: 410d2b6
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Sun Aug 7 01:11:14 2016 -0500
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Sat Aug 6 23:29:11 2016 -0700

----------------------------------------------------------------------
 CMakeLists.txt                                  | 38 ++++++++++----------
 CREDITS.md                                      |  2 +-
 GENERATOR_FUNCTIONS.md                          |  6 ++--
 NOTICE                                          |  2 +-
 README.md                                       | 18 +++++-----
 WORKING_WITH_AN_IDE.md                          | 18 +++++-----
 build/profile_build.sh                          | 23 +++++++-----
 build/vagrant/debian-jessie-amd64/Vagrantfile   | 19 ++++++++--
 build/vagrant/freebsd-10.2-amd64/Vagrantfile    | 19 ++++++++--
 build/vagrant/ubuntu-precise-amd64/Vagrantfile  | 19 ++++++++--
 catalog/CMakeLists.txt                          | 28 +++++++--------
 catalog/Catalog.cpp                             | 26 +++++++-------
 catalog/Catalog.hpp                             | 26 +++++++-------
 catalog/Catalog.proto                           | 30 ++++++++--------
 catalog/CatalogAttribute.cpp                    | 26 +++++++-------
 catalog/CatalogAttribute.hpp                    | 26 +++++++-------
 catalog/CatalogConfig.h.in                      | 26 +++++++-------
 catalog/CatalogDatabase.cpp                     | 26 +++++++-------
 catalog/CatalogDatabase.hpp                     | 26 +++++++-------
 catalog/CatalogDatabaseCache.cpp                | 25 +++++++------
 catalog/CatalogDatabaseCache.hpp                | 25 +++++++------
 catalog/CatalogDatabaseLite.hpp                 | 25 +++++++------
 catalog/CatalogErrors.hpp                       | 26 +++++++-------
 catalog/CatalogModule.hpp                       | 26 +++++++-------
 catalog/CatalogRelation.cpp                     | 28 +++++++--------
 catalog/CatalogRelation.hpp                     | 28 +++++++--------
 catalog/CatalogRelationSchema.cpp               | 28 +++++++--------
 catalog/CatalogRelationSchema.hpp               | 28 +++++++--------
 catalog/CatalogRelationStatistics.cpp           | 26 +++++++-------
 catalog/CatalogRelationStatistics.hpp           | 26 +++++++-------
 catalog/CatalogTypedefs.hpp                     | 28 +++++++--------
 catalog/IndexScheme.cpp                         | 26 +++++++-------
 catalog/IndexScheme.hpp                         | 26 +++++++-------
 catalog/NUMAPlacementScheme.cpp                 | 26 +++++++-------
 catalog/NUMAPlacementScheme.hpp                 | 26 +++++++-------
 catalog/PartitionScheme.cpp                     | 26 +++++++-------
 catalog/PartitionScheme.hpp                     | 26 +++++++-------
 catalog/PartitionSchemeHeader.cpp               | 26 +++++++-------
 catalog/PartitionSchemeHeader.hpp               | 26 +++++++-------
 catalog/tests/Catalog_unittest.cpp              | 28 +++++++--------
 catalog/tests/NUMAPlacementScheme_unittest.cpp  | 26 +++++++-------
 catalog/tests/PartitionScheme_unittest.cpp      | 26 +++++++-------
 cli/CMakeLists.txt                              | 28 +++++++--------
 cli/CliConfig.h.in                              | 26 +++++++-------
 cli/CliModule.hpp                               | 26 +++++++-------
 cli/CommandExecutor.cpp                         | 26 +++++++-------
 cli/CommandExecutor.hpp                         | 26 +++++++-------
 cli/DefaultsConfigurator.hpp                    | 26 +++++++-------
 cli/DropRelation.cpp                            | 25 +++++++------
 cli/DropRelation.hpp                            | 25 +++++++------
 cli/InputParserUtil.cpp                         | 25 +++++++------
 cli/InputParserUtil.hpp                         | 25 +++++++------
 cli/LineReader.cpp                              | 26 +++++++-------
 cli/LineReader.hpp                              | 26 +++++++-------
 cli/LineReaderDumb.cpp                          | 26 +++++++-------
 cli/LineReaderDumb.hpp                          | 26 +++++++-------
 cli/LineReaderLineNoise.cpp                     | 26 +++++++-------
 cli/LineReaderLineNoise.hpp                     | 26 +++++++-------
 cli/PrintToScreen.cpp                           | 26 +++++++-------
 cli/PrintToScreen.hpp                           | 26 +++++++-------
 cli/QuickstepCli.cpp                            | 28 +++++++--------
 cli/tests/CMakeLists.txt                        | 28 ++++++++-------
 cli/tests/CommandExecutorTest.cpp               | 26 +++++++-------
 cli/tests/CommandExecutorTestRunner.cpp         | 26 +++++++-------
 cli/tests/CommandExecutorTestRunner.hpp         | 26 +++++++-------
 cli/tests/command_executor/CMakeLists.txt       | 26 +++++++-------
 cli/tests/command_executor/D.test               | 26 +++++++-------
 cli/tests/command_executor/Dt.test              | 26 +++++++-------
 cmake/FindGSasl.cmake                           | 26 +++++++-------
 cmake/FindKerberos.cmake                        | 26 +++++++-------
 cmake/FindLibNuma.cmake                         | 26 +++++++-------
 cmake/FindLibhdfs3.cmake                        | 26 +++++++-------
 compression/CMakeLists.txt                      | 26 +++++++-------
 compression/CompressionDictionary.cpp           | 26 +++++++-------
 compression/CompressionDictionary.hpp           | 26 +++++++-------
 compression/CompressionDictionaryBuilder.cpp    | 26 +++++++-------
 compression/CompressionDictionaryBuilder.hpp    | 26 +++++++-------
 compression/CompressionDictionaryLite.cpp       | 26 +++++++-------
 compression/CompressionDictionaryLite.hpp       | 26 +++++++-------
 compression/CompressionModule.hpp               | 26 +++++++-------
 .../tests/CompressionDictionary_unittest.cpp    | 26 +++++++-------
 cyclic_dependency.py                            | 26 +++++++-------
 empty_src.cpp                                   | 25 +++++++------
 expressions/CMakeLists.txt                      | 26 +++++++-------
 expressions/ExpressionFactories.cpp             | 26 +++++++-------
 expressions/ExpressionFactories.hpp             | 26 +++++++-------
 expressions/Expressions.proto                   | 26 +++++++-------
 expressions/ExpressionsModule.hpp               | 26 +++++++-------
 expressions/aggregation/AggregateFunction.cpp   | 25 +++++++------
 expressions/aggregation/AggregateFunction.hpp   | 27 +++++++-------
 expressions/aggregation/AggregateFunction.proto | 25 +++++++------
 .../aggregation/AggregateFunctionAvg.cpp        | 25 +++++++------
 .../aggregation/AggregateFunctionAvg.hpp        | 25 +++++++------
 .../aggregation/AggregateFunctionCount.cpp      | 25 +++++++------
 .../aggregation/AggregateFunctionCount.hpp      | 25 +++++++------
 .../aggregation/AggregateFunctionFactory.cpp    | 25 +++++++------
 .../aggregation/AggregateFunctionFactory.hpp    | 25 +++++++------
 .../aggregation/AggregateFunctionMax.cpp        | 25 +++++++------
 .../aggregation/AggregateFunctionMax.hpp        | 25 +++++++------
 .../aggregation/AggregateFunctionMin.cpp        | 25 +++++++------
 .../aggregation/AggregateFunctionMin.hpp        | 25 +++++++------
 .../aggregation/AggregateFunctionSum.cpp        | 25 +++++++------
 .../aggregation/AggregateFunctionSum.hpp        | 25 +++++++------
 .../aggregation/AggregationConcreteHandle.cpp   | 26 +++++++-------
 .../aggregation/AggregationConcreteHandle.hpp   | 28 +++++++--------
 expressions/aggregation/AggregationHandle.hpp   | 28 +++++++--------
 .../aggregation/AggregationHandleAvg.cpp        | 28 +++++++--------
 .../aggregation/AggregationHandleAvg.hpp        | 28 +++++++--------
 .../aggregation/AggregationHandleCount.cpp      | 28 +++++++--------
 .../aggregation/AggregationHandleCount.hpp      | 28 +++++++--------
 .../aggregation/AggregationHandleDistinct.cpp   | 26 +++++++-------
 .../aggregation/AggregationHandleDistinct.hpp   | 26 +++++++-------
 .../aggregation/AggregationHandleMax.cpp        | 26 +++++++-------
 .../aggregation/AggregationHandleMax.hpp        | 28 +++++++--------
 .../aggregation/AggregationHandleMin.cpp        | 28 +++++++--------
 .../aggregation/AggregationHandleMin.hpp        | 28 +++++++--------
 .../aggregation/AggregationHandleSum.cpp        | 28 +++++++--------
 .../aggregation/AggregationHandleSum.hpp        | 28 +++++++--------
 expressions/aggregation/AggregationID.hpp       | 26 +++++++-------
 expressions/aggregation/CMakeLists.txt          | 26 +++++++-------
 .../tests/AggregationHandleAvg_unittest.cpp     | 26 +++++++-------
 .../tests/AggregationHandleCount_unittest.cpp   | 26 +++++++-------
 .../tests/AggregationHandleMax_unittest.cpp     | 26 +++++++-------
 .../tests/AggregationHandleMin_unittest.cpp     | 26 +++++++-------
 .../tests/AggregationHandleSum_unittest.cpp     | 26 +++++++-------
 expressions/predicate/CMakeLists.txt            | 26 +++++++-------
 expressions/predicate/ComparisonPredicate.cpp   | 26 +++++++-------
 expressions/predicate/ComparisonPredicate.hpp   | 26 +++++++-------
 expressions/predicate/ConjunctionPredicate.cpp  | 26 +++++++-------
 expressions/predicate/ConjunctionPredicate.hpp  | 26 +++++++-------
 expressions/predicate/DisjunctionPredicate.cpp  | 26 +++++++-------
 expressions/predicate/DisjunctionPredicate.hpp  | 26 +++++++-------
 expressions/predicate/NegationPredicate.cpp     | 26 +++++++-------
 expressions/predicate/NegationPredicate.hpp     | 26 +++++++-------
 expressions/predicate/Predicate.cpp             | 26 +++++++-------
 expressions/predicate/Predicate.hpp             | 26 +++++++-------
 expressions/predicate/PredicateCost.hpp         | 26 +++++++-------
 expressions/predicate/PredicateWithList.hpp     | 26 +++++++-------
 expressions/predicate/TrivialPredicates.hpp     | 26 +++++++-------
 .../predicate/tests/Predicate_unittest.cpp      | 26 +++++++-------
 expressions/scalar/CMakeLists.txt               | 26 +++++++-------
 expressions/scalar/Scalar.cpp                   | 26 +++++++-------
 expressions/scalar/Scalar.hpp                   | 26 +++++++-------
 expressions/scalar/ScalarAttribute.cpp          | 26 +++++++-------
 expressions/scalar/ScalarAttribute.hpp          | 26 +++++++-------
 expressions/scalar/ScalarBinaryExpression.cpp   | 26 +++++++-------
 expressions/scalar/ScalarBinaryExpression.hpp   | 26 +++++++-------
 expressions/scalar/ScalarCaseExpression.cpp     | 26 +++++++-------
 expressions/scalar/ScalarCaseExpression.hpp     | 26 +++++++-------
 expressions/scalar/ScalarLiteral.cpp            | 26 +++++++-------
 expressions/scalar/ScalarLiteral.hpp            | 26 +++++++-------
 expressions/scalar/ScalarUnaryExpression.cpp    | 26 +++++++-------
 expressions/scalar/ScalarUnaryExpression.hpp    | 26 +++++++-------
 .../tests/ScalarCaseExpression_unittest.cpp     | 26 +++++++-------
 expressions/scalar/tests/Scalar_unittest.cpp    | 28 +++++++--------
 expressions/table_generator/CMakeLists.txt      | 25 +++++++------
 expressions/table_generator/GenerateSeries.hpp  | 27 +++++++-------
 .../table_generator/GenerateSeriesHandle.hpp    | 27 +++++++-------
 .../table_generator/GeneratorFunction.hpp       | 27 +++++++-------
 .../table_generator/GeneratorFunction.proto     | 25 +++++++------
 .../GeneratorFunctionFactory.cpp                | 27 +++++++-------
 .../GeneratorFunctionFactory.hpp                | 27 +++++++-------
 .../table_generator/GeneratorFunctionHandle.hpp | 27 +++++++-------
 expressions/window_aggregation/CMakeLists.txt   | 32 ++++++++---------
 .../WindowAggregateFunction.cpp                 | 32 ++++++++---------
 .../WindowAggregateFunction.hpp                 | 36 +++++++++----------
 .../WindowAggregateFunction.proto               | 34 +++++++++---------
 .../WindowAggregateFunctionAvg.cpp              | 32 ++++++++---------
 .../WindowAggregateFunctionAvg.hpp              | 32 ++++++++---------
 .../WindowAggregateFunctionCount.cpp            | 32 ++++++++---------
 .../WindowAggregateFunctionCount.hpp            | 32 ++++++++---------
 .../WindowAggregateFunctionFactory.cpp          | 32 ++++++++---------
 .../WindowAggregateFunctionFactory.hpp          | 32 ++++++++---------
 .../WindowAggregateFunctionMax.cpp              | 32 ++++++++---------
 .../WindowAggregateFunctionMax.hpp              | 32 ++++++++---------
 .../WindowAggregateFunctionMin.cpp              | 32 ++++++++---------
 .../WindowAggregateFunctionMin.hpp              | 32 ++++++++---------
 .../WindowAggregateFunctionSum.cpp              | 32 ++++++++---------
 .../WindowAggregateFunctionSum.hpp              | 32 ++++++++---------
 .../WindowAggregationHandle.cpp                 | 32 ++++++++---------
 .../WindowAggregationHandle.hpp                 | 32 ++++++++---------
 .../WindowAggregationHandleAvg.cpp              | 32 ++++++++---------
 .../WindowAggregationHandleAvg.hpp              | 32 ++++++++---------
 .../window_aggregation/WindowAggregationID.hpp  | 32 ++++++++---------
 .../WindowAggregationHandleAvg_unittest.cpp     | 33 +++++++++--------
 parser/CMakeLists.txt                           | 28 +++++++--------
 parser/ParseAssignment.hpp                      | 26 +++++++-------
 parser/ParseAttributeDefinition.cpp             | 26 +++++++-------
 parser/ParseAttributeDefinition.hpp             | 26 +++++++-------
 parser/ParseBasicExpressions.cpp                | 28 +++++++--------
 parser/ParseBasicExpressions.hpp                | 28 +++++++--------
 parser/ParseBlockProperties.cpp                 | 26 +++++++-------
 parser/ParseBlockProperties.hpp                 | 26 +++++++-------
 parser/ParseCaseExpressions.cpp                 | 26 +++++++-------
 parser/ParseCaseExpressions.hpp                 | 26 +++++++-------
 parser/ParseExpression.hpp                      | 28 +++++++--------
 parser/ParseGeneratorTableReference.cpp         | 27 +++++++-------
 parser/ParseGeneratorTableReference.hpp         | 27 +++++++-------
 parser/ParseGroupBy.cpp                         | 26 +++++++-------
 parser/ParseGroupBy.hpp                         | 26 +++++++-------
 parser/ParseHaving.cpp                          | 26 +++++++-------
 parser/ParseHaving.hpp                          | 26 +++++++-------
 parser/ParseIndexProperties.cpp                 | 26 +++++++-------
 parser/ParseIndexProperties.hpp                 | 26 +++++++-------
 parser/ParseJoinedTableReference.cpp            | 26 +++++++-------
 parser/ParseJoinedTableReference.hpp            | 26 +++++++-------
 parser/ParseKeyValue.hpp                        | 25 +++++++------
 parser/ParseLimit.cpp                           | 26 +++++++-------
 parser/ParseLimit.hpp                           | 26 +++++++-------
 parser/ParseLiteralValue.cpp                    | 26 +++++++-------
 parser/ParseLiteralValue.hpp                    | 26 +++++++-------
 parser/ParseOrderBy.cpp                         | 26 +++++++-------
 parser/ParseOrderBy.hpp                         | 26 +++++++-------
 parser/ParsePartitionClause.hpp                 | 26 +++++++-------
 parser/ParsePredicate.cpp                       | 28 +++++++--------
 parser/ParsePredicate.hpp                       | 28 +++++++--------
 parser/ParsePredicateExists.hpp                 | 26 +++++++-------
 parser/ParsePredicateInTableQuery.hpp           | 26 +++++++-------
 parser/ParsePriority.hpp                        | 26 +++++++-------
 parser/ParseSample.cpp                          | 26 +++++++-------
 parser/ParseSample.hpp                          | 26 +++++++-------
 parser/ParseSelect.hpp                          | 28 +++++++--------
 parser/ParseSelectionClause.cpp                 | 26 +++++++-------
 parser/ParseSelectionClause.hpp                 | 26 +++++++-------
 parser/ParseSimpleTableReference.cpp            | 26 +++++++-------
 parser/ParseSimpleTableReference.hpp            | 30 ++++++++--------
 parser/ParseStatement.hpp                       | 28 +++++++--------
 parser/ParseString.cpp                          | 26 +++++++-------
 parser/ParseString.hpp                          | 26 +++++++-------
 parser/ParseSubqueryExpression.cpp              | 28 +++++++--------
 parser/ParseSubqueryExpression.hpp              | 26 +++++++-------
 parser/ParseSubqueryTableReference.cpp          | 26 +++++++-------
 parser/ParseSubqueryTableReference.hpp          | 26 +++++++-------
 parser/ParseTableReference.cpp                  | 26 +++++++-------
 parser/ParseTableReference.hpp                  | 26 +++++++-------
 parser/ParseTreeNode.hpp                        | 26 +++++++-------
 parser/ParseWindow.hpp                          | 28 ++++++++-------
 parser/ParserModule.hpp                         | 26 +++++++-------
 parser/ParserUtil.cpp                           | 26 +++++++-------
 parser/ParserUtil.hpp                           | 26 +++++++-------
 parser/SqlLexer.lpp                             | 28 +++++++--------
 parser/SqlParser.ypp                            | 34 +++++++++---------
 parser/SqlParserWrapper.cpp                     | 26 +++++++-------
 parser/SqlParserWrapper.hpp                     | 26 +++++++-------
 parser/preprocessed/SqlLexer_gen.cpp            | 28 +++++++--------
 parser/tests/Aggregate.test                     | 26 +++++++-------
 parser/tests/Alter.test                         | 26 +++++++-------
 parser/tests/CMakeLists.txt                     | 26 +++++++-------
 parser/tests/Copy.test                          | 26 +++++++-------
 parser/tests/Create.test                        | 26 +++++++-------
 parser/tests/Delete.test                        | 26 +++++++-------
 parser/tests/Drop.test                          | 26 +++++++-------
 parser/tests/Index.test                         | 26 +++++++-------
 parser/tests/Insert.test                        | 26 +++++++-------
 parser/tests/Join.test                          | 26 +++++++-------
 parser/tests/ParserTest.cpp                     | 26 +++++++-------
 parser/tests/Select.test                        | 36 +++++++++----------
 parser/tests/TPCH.test                          | 28 +++++++--------
 parser/tests/Update.test                        | 26 +++++++-------
 query_execution/AdmitRequestMessage.hpp         | 26 +++++++-------
 query_execution/BlockLocator.cpp                | 25 +++++++------
 query_execution/BlockLocator.hpp                | 25 +++++++------
 query_execution/CMakeLists.txt                  | 28 +++++++--------
 query_execution/ForemanBase.hpp                 | 25 +++++++------
 query_execution/ForemanSingleNode.cpp           | 26 +++++++-------
 query_execution/ForemanSingleNode.hpp           | 26 +++++++-------
 query_execution/PolicyEnforcerBase.cpp          | 26 +++++++-------
 query_execution/PolicyEnforcerBase.hpp          | 26 +++++++-------
 query_execution/PolicyEnforcerSingleNode.cpp    | 23 +++++++-----
 query_execution/PolicyEnforcerSingleNode.hpp    | 23 +++++++-----
 query_execution/QueryContext.cpp                | 28 +++++++--------
 query_execution/QueryContext.hpp                | 28 +++++++--------
 query_execution/QueryContext.proto              | 26 +++++++-------
 query_execution/QueryExecutionMessages.proto    | 29 ++++++++-------
 query_execution/QueryExecutionModule.hpp        | 26 +++++++-------
 query_execution/QueryExecutionState.hpp         | 26 +++++++-------
 query_execution/QueryExecutionTypedefs.hpp      | 26 +++++++-------
 query_execution/QueryExecutionUtil.hpp          | 25 +++++++------
 query_execution/QueryManagerBase.cpp            | 26 +++++++-------
 query_execution/QueryManagerBase.hpp            | 26 +++++++-------
 query_execution/QueryManagerDistributed.cpp     | 23 +++++++-----
 query_execution/QueryManagerDistributed.hpp     | 23 +++++++-----
 query_execution/QueryManagerSingleNode.cpp      | 26 +++++++-------
 query_execution/QueryManagerSingleNode.hpp      | 26 +++++++-------
 query_execution/Shiftboss.cpp                   | 23 +++++++-----
 query_execution/Shiftboss.hpp                   | 23 +++++++-----
 query_execution/ShiftbossDirectory.hpp          | 23 +++++++-----
 query_execution/WorkOrderProtosContainer.hpp    | 25 +++++++------
 query_execution/WorkOrdersContainer.cpp         | 26 +++++++-------
 query_execution/WorkOrdersContainer.hpp         | 26 +++++++-------
 query_execution/Worker.cpp                      | 25 +++++++------
 query_execution/Worker.hpp                      | 26 +++++++-------
 query_execution/WorkerDirectory.hpp             | 25 +++++++------
 query_execution/WorkerMessage.hpp               | 26 +++++++-------
 query_execution/WorkerSelectionPolicy.hpp       | 25 +++++++------
 query_execution/tests/BlockLocator_unittest.cpp | 25 +++++++------
 .../tests/QueryManagerSingleNode_unittest.cpp   | 26 +++++++-------
 .../tests/WorkOrdersContainer_unittest.cpp      | 25 +++++++------
 .../tests/WorkerDirectory_unittest.cpp          | 25 +++++++------
 .../tests/WorkerSelectionPolicy_unittest.cpp    | 25 +++++++------
 query_optimizer/CMakeLists.txt                  | 28 +++++++--------
 query_optimizer/ExecutionGenerator.cpp          | 28 +++++++--------
 query_optimizer/ExecutionGenerator.hpp          | 28 +++++++--------
 query_optimizer/ExecutionHeuristics.cpp         | 26 +++++++-------
 query_optimizer/ExecutionHeuristics.hpp         | 26 +++++++-------
 query_optimizer/LogicalGenerator.cpp            | 28 +++++++--------
 query_optimizer/LogicalGenerator.hpp            | 26 +++++++-------
 query_optimizer/LogicalToPhysicalMapper.hpp     | 25 +++++++------
 query_optimizer/Optimizer.cpp                   | 26 +++++++-------
 query_optimizer/Optimizer.hpp                   | 26 +++++++-------
 query_optimizer/OptimizerContext.cpp            | 26 +++++++-------
 query_optimizer/OptimizerContext.hpp            | 26 +++++++-------
 query_optimizer/OptimizerTree.hpp               | 26 +++++++-------
 query_optimizer/PhysicalGenerator.cpp           | 28 +++++++--------
 query_optimizer/PhysicalGenerator.hpp           | 26 +++++++-------
 query_optimizer/QueryHandle.hpp                 | 25 +++++++------
 query_optimizer/QueryOptimizerConfig.h.in       | 25 +++++++------
 query_optimizer/QueryOptimizerModule.hpp        | 26 +++++++-------
 query_optimizer/QueryPlan.hpp                   | 26 +++++++-------
 query_optimizer/QueryProcessor.cpp              | 26 +++++++-------
 query_optimizer/QueryProcessor.hpp              | 28 +++++++--------
 query_optimizer/Validator.hpp                   | 26 +++++++-------
 query_optimizer/cost_model/CMakeLists.txt       | 28 +++++++--------
 query_optimizer/cost_model/CostModel.hpp        | 26 +++++++-------
 query_optimizer/cost_model/CostModelModule.hpp  | 26 +++++++-------
 query_optimizer/cost_model/SimpleCostModel.cpp  | 28 +++++++--------
 query_optimizer/cost_model/SimpleCostModel.hpp  | 28 +++++++--------
 .../cost_model/StarSchemaSimpleCostModel.cpp    | 26 +++++++-------
 .../cost_model/StarSchemaSimpleCostModel.hpp    | 26 +++++++-------
 .../expressions/AggregateFunction.cpp           | 27 +++++++-------
 .../expressions/AggregateFunction.hpp           | 28 +++++++--------
 query_optimizer/expressions/Alias.cpp           | 26 +++++++-------
 query_optimizer/expressions/Alias.hpp           | 26 +++++++-------
 .../expressions/AttributeReference.cpp          | 28 +++++++--------
 .../expressions/AttributeReference.hpp          | 28 +++++++--------
 .../expressions/BinaryExpression.cpp            | 28 +++++++--------
 .../expressions/BinaryExpression.hpp            | 26 +++++++-------
 query_optimizer/expressions/CMakeLists.txt      | 28 +++++++--------
 query_optimizer/expressions/Cast.cpp            | 26 +++++++-------
 query_optimizer/expressions/Cast.hpp            | 26 +++++++-------
 .../expressions/ComparisonExpression.cpp        | 28 +++++++--------
 .../expressions/ComparisonExpression.hpp        | 26 +++++++-------
 query_optimizer/expressions/Exists.cpp          | 26 +++++++-------
 query_optimizer/expressions/Exists.hpp          | 26 +++++++-------
 query_optimizer/expressions/ExprId.hpp          | 26 +++++++-------
 query_optimizer/expressions/Expression.hpp      | 26 +++++++-------
 query_optimizer/expressions/ExpressionType.hpp  | 28 +++++++--------
 query_optimizer/expressions/ExpressionUtil.cpp  | 28 +++++++--------
 query_optimizer/expressions/ExpressionUtil.hpp  | 28 +++++++--------
 query_optimizer/expressions/InTableQuery.cpp    | 26 +++++++-------
 query_optimizer/expressions/InTableQuery.hpp    | 26 +++++++-------
 query_optimizer/expressions/InValueList.cpp     | 26 +++++++-------
 query_optimizer/expressions/InValueList.hpp     | 26 +++++++-------
 query_optimizer/expressions/LogicalAnd.cpp      | 26 +++++++-------
 query_optimizer/expressions/LogicalAnd.hpp      | 26 +++++++-------
 query_optimizer/expressions/LogicalNot.cpp      | 26 +++++++-------
 query_optimizer/expressions/LogicalNot.hpp      | 26 +++++++-------
 query_optimizer/expressions/LogicalOr.cpp       | 26 +++++++-------
 query_optimizer/expressions/LogicalOr.hpp       | 26 +++++++-------
 query_optimizer/expressions/NamedExpression.cpp | 26 +++++++-------
 query_optimizer/expressions/NamedExpression.hpp | 26 +++++++-------
 .../expressions/OptimizerExpressionsModule.hpp  | 26 +++++++-------
 query_optimizer/expressions/PatternMatcher.hpp  | 28 +++++++--------
 query_optimizer/expressions/Predicate.hpp       | 26 +++++++-------
 .../expressions/PredicateLiteral.cpp            | 26 +++++++-------
 .../expressions/PredicateLiteral.hpp            | 26 +++++++-------
 query_optimizer/expressions/Scalar.hpp          | 26 +++++++-------
 query_optimizer/expressions/ScalarLiteral.cpp   | 26 +++++++-------
 query_optimizer/expressions/ScalarLiteral.hpp   | 26 +++++++-------
 query_optimizer/expressions/SearchedCase.cpp    | 26 +++++++-------
 query_optimizer/expressions/SearchedCase.hpp    | 26 +++++++-------
 query_optimizer/expressions/SimpleCase.cpp      | 26 +++++++-------
 query_optimizer/expressions/SimpleCase.hpp      | 26 +++++++-------
 .../expressions/SubqueryExpression.cpp          | 26 +++++++-------
 .../expressions/SubqueryExpression.hpp          | 26 +++++++-------
 query_optimizer/expressions/UnaryExpression.cpp | 26 +++++++-------
 query_optimizer/expressions/UnaryExpression.hpp | 26 +++++++-------
 .../expressions/WindowAggregateFunction.cpp     | 27 +++++++-------
 .../expressions/WindowAggregateFunction.hpp     | 28 +++++++--------
 query_optimizer/logical/Aggregate.cpp           | 28 +++++++--------
 query_optimizer/logical/Aggregate.hpp           | 28 +++++++--------
 query_optimizer/logical/BinaryJoin.cpp          | 26 +++++++-------
 query_optimizer/logical/BinaryJoin.hpp          | 26 +++++++-------
 query_optimizer/logical/CMakeLists.txt          | 30 ++++++++--------
 query_optimizer/logical/CopyFrom.cpp            | 26 +++++++-------
 query_optimizer/logical/CopyFrom.hpp            | 26 +++++++-------
 query_optimizer/logical/CreateIndex.cpp         | 26 +++++++-------
 query_optimizer/logical/CreateIndex.hpp         | 26 +++++++-------
 query_optimizer/logical/CreateTable.cpp         | 26 +++++++-------
 query_optimizer/logical/CreateTable.hpp         | 26 +++++++-------
 query_optimizer/logical/DeleteTuples.cpp        | 26 +++++++-------
 query_optimizer/logical/DeleteTuples.hpp        | 26 +++++++-------
 query_optimizer/logical/DropTable.cpp           | 26 +++++++-------
 query_optimizer/logical/DropTable.hpp           | 26 +++++++-------
 query_optimizer/logical/Filter.cpp              | 29 +++++++--------
 query_optimizer/logical/Filter.hpp              | 28 +++++++--------
 query_optimizer/logical/HashJoin.hpp            | 28 +++++++--------
 query_optimizer/logical/InsertSelection.cpp     | 26 +++++++-------
 query_optimizer/logical/InsertSelection.hpp     | 26 +++++++-------
 query_optimizer/logical/InsertTuple.cpp         | 26 +++++++-------
 query_optimizer/logical/InsertTuple.hpp         | 26 +++++++-------
 query_optimizer/logical/Join.hpp                | 26 +++++++-------
 query_optimizer/logical/Logical.hpp             | 28 +++++++--------
 query_optimizer/logical/LogicalType.hpp         | 26 +++++++-------
 .../logical/MultiwayCartesianJoin.cpp           | 26 +++++++-------
 .../logical/MultiwayCartesianJoin.hpp           | 26 +++++++-------
 query_optimizer/logical/NestedLoopsJoin.hpp     | 28 +++++++--------
 .../logical/OptimizerLogicalModule.hpp          | 26 +++++++-------
 query_optimizer/logical/PatternMatcher.hpp      | 26 +++++++-------
 query_optimizer/logical/Project.cpp             | 28 +++++++--------
 query_optimizer/logical/Project.hpp             | 28 +++++++--------
 query_optimizer/logical/Sample.cpp              | 26 +++++++-------
 query_optimizer/logical/Sample.hpp              | 30 ++++++++--------
 .../logical/SharedSubplanReference.cpp          | 28 +++++++--------
 .../logical/SharedSubplanReference.hpp          | 28 +++++++--------
 query_optimizer/logical/Sort.cpp                | 26 +++++++-------
 query_optimizer/logical/Sort.hpp                | 26 +++++++-------
 query_optimizer/logical/TableGenerator.hpp      | 27 +++++++-------
 query_optimizer/logical/TableReference.cpp      | 26 +++++++-------
 query_optimizer/logical/TableReference.hpp      | 26 +++++++-------
 query_optimizer/logical/TopLevelPlan.cpp        | 26 +++++++-------
 query_optimizer/logical/TopLevelPlan.hpp        | 28 +++++++--------
 query_optimizer/logical/UpdateTable.cpp         | 26 +++++++-------
 query_optimizer/logical/UpdateTable.hpp         | 26 +++++++-------
 query_optimizer/logical/WindowAggregate.cpp     | 28 +++++++--------
 query_optimizer/logical/WindowAggregate.hpp     | 28 +++++++--------
 query_optimizer/physical/Aggregate.cpp          | 28 +++++++--------
 query_optimizer/physical/Aggregate.hpp          | 26 +++++++-------
 query_optimizer/physical/BinaryJoin.cpp         | 26 +++++++-------
 query_optimizer/physical/BinaryJoin.hpp         | 26 +++++++-------
 query_optimizer/physical/CMakeLists.txt         | 26 +++++++-------
 query_optimizer/physical/CopyFrom.cpp           | 26 +++++++-------
 query_optimizer/physical/CopyFrom.hpp           | 26 +++++++-------
 query_optimizer/physical/CreateIndex.cpp        | 26 +++++++-------
 query_optimizer/physical/CreateIndex.hpp        | 26 +++++++-------
 query_optimizer/physical/CreateTable.cpp        | 26 +++++++-------
 query_optimizer/physical/CreateTable.hpp        | 26 +++++++-------
 query_optimizer/physical/DeleteTuples.cpp       | 26 +++++++-------
 query_optimizer/physical/DeleteTuples.hpp       | 26 +++++++-------
 query_optimizer/physical/DropTable.cpp          | 26 +++++++-------
 query_optimizer/physical/DropTable.hpp          | 26 +++++++-------
 query_optimizer/physical/HashJoin.cpp           | 28 +++++++--------
 query_optimizer/physical/HashJoin.hpp           | 28 +++++++--------
 query_optimizer/physical/InsertSelection.cpp    | 26 +++++++-------
 query_optimizer/physical/InsertSelection.hpp    | 26 +++++++-------
 query_optimizer/physical/InsertTuple.cpp        | 26 +++++++-------
 query_optimizer/physical/InsertTuple.hpp        | 26 +++++++-------
 query_optimizer/physical/Join.hpp               | 26 +++++++-------
 query_optimizer/physical/NestedLoopsJoin.cpp    | 26 +++++++-------
 query_optimizer/physical/NestedLoopsJoin.hpp    | 26 +++++++-------
 .../physical/OptimizerPhysicalModule.hpp        | 26 +++++++-------
 query_optimizer/physical/PatternMatcher.hpp     | 26 +++++++-------
 query_optimizer/physical/Physical.hpp           | 26 +++++++-------
 query_optimizer/physical/PhysicalType.hpp       | 26 +++++++-------
 query_optimizer/physical/Sample.cpp             | 26 +++++++-------
 query_optimizer/physical/Sample.hpp             | 30 ++++++++--------
 query_optimizer/physical/Selection.cpp          | 26 +++++++-------
 query_optimizer/physical/Selection.hpp          | 26 +++++++-------
 .../physical/SharedSubplanReference.cpp         | 28 +++++++--------
 .../physical/SharedSubplanReference.hpp         | 28 +++++++--------
 query_optimizer/physical/Sort.cpp               | 26 +++++++-------
 query_optimizer/physical/Sort.hpp               | 26 +++++++-------
 query_optimizer/physical/TableGenerator.hpp     | 27 +++++++-------
 query_optimizer/physical/TableReference.cpp     | 26 +++++++-------
 query_optimizer/physical/TableReference.hpp     | 26 +++++++-------
 query_optimizer/physical/TopLevelPlan.cpp       | 26 +++++++-------
 query_optimizer/physical/TopLevelPlan.hpp       | 28 +++++++--------
 query_optimizer/physical/UpdateTable.cpp        | 26 +++++++-------
 query_optimizer/physical/UpdateTable.hpp        | 26 +++++++-------
 query_optimizer/physical/WindowAggregate.cpp    | 28 +++++++--------
 query_optimizer/physical/WindowAggregate.hpp    | 28 +++++++--------
 query_optimizer/resolver/CMakeLists.txt         | 28 +++++++--------
 query_optimizer/resolver/NameResolver.cpp       | 28 +++++++--------
 query_optimizer/resolver/NameResolver.hpp       | 28 +++++++--------
 .../resolver/QueryResolverModule.hpp            | 26 +++++++-------
 query_optimizer/resolver/Resolver.cpp           | 28 +++++++--------
 query_optimizer/resolver/Resolver.hpp           | 28 ++++++++-------
 query_optimizer/rules/BottomUpRule.hpp          | 26 +++++++-------
 query_optimizer/rules/CMakeLists.txt            | 28 +++++++--------
 query_optimizer/rules/CollapseProject.cpp       | 26 +++++++-------
 query_optimizer/rules/CollapseProject.hpp       | 26 +++++++-------
 query_optimizer/rules/GenerateJoins.cpp         | 28 +++++++--------
 query_optimizer/rules/GenerateJoins.hpp         | 26 +++++++-------
 query_optimizer/rules/OptimizerRulesModule.hpp  | 26 +++++++-------
 query_optimizer/rules/PruneColumns.cpp          | 26 +++++++-------
 query_optimizer/rules/PruneColumns.hpp          | 26 +++++++-------
 query_optimizer/rules/PushDownFilter.cpp        | 28 +++++++--------
 query_optimizer/rules/PushDownFilter.hpp        | 26 +++++++-------
 query_optimizer/rules/PushDownSemiAntiJoin.cpp  | 26 +++++++-------
 query_optimizer/rules/PushDownSemiAntiJoin.hpp  | 26 +++++++-------
 query_optimizer/rules/Rule.hpp                  | 26 +++++++-------
 query_optimizer/rules/RuleHelper.cpp            | 28 +++++++--------
 query_optimizer/rules/RuleHelper.hpp            | 26 +++++++-------
 .../StarSchemaHashJoinOrderOptimization.cpp     | 26 +++++++-------
 .../StarSchemaHashJoinOrderOptimization.hpp     | 26 +++++++-------
 query_optimizer/rules/SwapProbeBuild.cpp        | 19 ++++++++++
 query_optimizer/rules/SwapProbeBuild.hpp        | 19 ++++++++++
 query_optimizer/rules/TopDownRule.hpp           | 26 +++++++-------
 query_optimizer/rules/UnnestSubqueries.cpp      | 26 +++++++-------
 query_optimizer/rules/UnnestSubqueries.hpp      | 26 +++++++-------
 query_optimizer/rules/UpdateExpression.cpp      | 26 +++++++-------
 query_optimizer/rules/UpdateExpression.hpp      | 26 +++++++-------
 query_optimizer/rules/tests/CMakeLists.txt      | 26 +++++++-------
 .../rules/tests/CollapseProject_unittest.cpp    | 26 +++++++-------
 .../rules/tests/ExpressionRuleTest.hpp          | 26 +++++++-------
 .../rules/tests/GenerateJoins_unittest.cpp      | 28 +++++++--------
 query_optimizer/rules/tests/LogicalRuleTest.hpp | 26 +++++++-------
 .../rules/tests/PhysicalRuleTest.hpp            | 26 +++++++-------
 .../rules/tests/PruneColumns_unittest.cpp       | 28 +++++++--------
 .../rules/tests/PushDownFilter_unittest.cpp     | 26 +++++++-------
 query_optimizer/rules/tests/RuleTest.hpp        | 26 +++++++-------
 .../rules/tests/UpdateExpression_unittest.cpp   | 26 +++++++-------
 query_optimizer/strategy/Aggregate.cpp          | 26 +++++++-------
 query_optimizer/strategy/Aggregate.hpp          | 26 +++++++-------
 query_optimizer/strategy/CMakeLists.txt         | 28 +++++++--------
 query_optimizer/strategy/Join.cpp               | 28 +++++++--------
 query_optimizer/strategy/Join.hpp               | 28 +++++++--------
 query_optimizer/strategy/OneToOne.cpp           | 28 +++++++--------
 query_optimizer/strategy/OneToOne.hpp           | 26 +++++++-------
 .../strategy/OptimizerStrategyModule.hpp        | 26 +++++++-------
 query_optimizer/strategy/Selection.cpp          | 26 +++++++-------
 query_optimizer/strategy/Selection.hpp          | 26 +++++++-------
 query_optimizer/strategy/Strategy.hpp           | 26 +++++++-------
 .../strategy/tests/Aggregate_unittest.cpp       | 28 +++++++--------
 query_optimizer/strategy/tests/CMakeLists.txt   | 26 +++++++-------
 .../strategy/tests/Join_unittest.cpp            | 28 +++++++--------
 .../strategy/tests/OneToOne_unittest.cpp        | 26 +++++++-------
 .../strategy/tests/Selection_unittest.cpp       | 26 +++++++-------
 query_optimizer/strategy/tests/StrategyTest.hpp | 26 +++++++-------
 query_optimizer/tests/CMakeLists.txt            | 26 +++++++-------
 .../tests/ExecutionGeneratorTest.cpp            | 26 +++++++-------
 .../tests/ExecutionGeneratorTestRunner.cpp      | 26 +++++++-------
 .../tests/ExecutionGeneratorTestRunner.hpp      | 26 +++++++-------
 .../tests/ExecutionHeuristics_unittest.cpp      | 26 +++++++-------
 query_optimizer/tests/OptimizerTest.cpp         | 26 +++++++-------
 query_optimizer/tests/OptimizerTest.hpp         | 26 +++++++-------
 query_optimizer/tests/OptimizerTextTest.cpp     | 26 +++++++-------
 .../tests/OptimizerTextTestRunner.cpp           | 26 +++++++-------
 .../tests/OptimizerTextTestRunner.hpp           | 26 +++++++-------
 query_optimizer/tests/TestDatabaseLoader.cpp    | 26 +++++++-------
 query_optimizer/tests/TestDatabaseLoader.hpp    | 26 +++++++-------
 .../tests/execution_generator/CMakeLists.txt    | 26 +++++++-------
 .../tests/execution_generator/Create.test       | 26 +++++++-------
 .../tests/execution_generator/Delete.test       | 26 +++++++-------
 .../tests/execution_generator/Distinct.test     | 26 +++++++-------
 .../tests/execution_generator/Drop.test         | 26 +++++++-------
 .../tests/execution_generator/Index.test        | 26 +++++++-------
 .../tests/execution_generator/Insert.test       | 26 +++++++-------
 .../tests/execution_generator/Join.test         | 26 +++++++-------
 .../tests/execution_generator/Select.test       | 28 +++++++--------
 .../StringPatternMatching.test                  | 26 +++++++-------
 .../execution_generator/TableGenerator.test     | 25 +++++++------
 .../tests/execution_generator/Update.test       | 26 +++++++-------
 .../tests/logical_generator/CMakeLists.txt      | 26 +++++++-------
 .../tests/logical_generator/Create.test         | 25 +++++++------
 .../tests/logical_generator/Index.test          | 26 +++++++-------
 .../tests/logical_generator/Join.test           | 26 +++++++-------
 .../tests/logical_generator/Select.test         | 28 +++++++--------
 .../tests/physical_generator/CMakeLists.txt     | 26 +++++++-------
 .../tests/physical_generator/Copy.test          | 26 +++++++-------
 .../tests/physical_generator/Create.test        | 26 +++++++-------
 .../tests/physical_generator/Delete.test        | 26 +++++++-------
 .../tests/physical_generator/Drop.test          | 26 +++++++-------
 .../tests/physical_generator/Index.test         | 26 +++++++-------
 .../tests/physical_generator/Insert.test        | 26 +++++++-------
 .../tests/physical_generator/Join.test          | 26 +++++++-------
 .../tests/physical_generator/Select.test        | 28 +++++++--------
 .../tests/physical_generator/Update.test        | 26 +++++++-------
 query_optimizer/tests/resolver/Aggregate.test   | 26 +++++++-------
 query_optimizer/tests/resolver/CMakeLists.txt   | 26 +++++++-------
 query_optimizer/tests/resolver/Copy.test        | 26 +++++++-------
 query_optimizer/tests/resolver/Create.test      | 26 +++++++-------
 query_optimizer/tests/resolver/Delete.test      | 26 +++++++-------
 query_optimizer/tests/resolver/Drop.test        | 26 +++++++-------
 query_optimizer/tests/resolver/Index.test       | 26 +++++++-------
 query_optimizer/tests/resolver/Insert.test      | 26 +++++++-------
 query_optimizer/tests/resolver/Join.test        | 26 +++++++-------
 query_optimizer/tests/resolver/Select.test      | 28 +++++++--------
 query_optimizer/tests/resolver/Update.test      | 26 +++++++-------
 relational_operators/AggregationOperator.cpp    | 26 +++++++-------
 relational_operators/AggregationOperator.hpp    | 26 +++++++-------
 relational_operators/BuildHashOperator.cpp      | 26 +++++++-------
 relational_operators/BuildHashOperator.hpp      | 26 +++++++-------
 relational_operators/CMakeLists.txt             | 30 ++++++++--------
 relational_operators/CreateIndexOperator.cpp    | 26 +++++++-------
 relational_operators/CreateIndexOperator.hpp    | 27 +++++++-------
 relational_operators/CreateTableOperator.cpp    | 26 +++++++-------
 relational_operators/CreateTableOperator.hpp    | 26 +++++++-------
 relational_operators/DeleteOperator.cpp         | 26 +++++++-------
 relational_operators/DeleteOperator.hpp         | 26 +++++++-------
 relational_operators/DestroyHashOperator.cpp    | 26 +++++++-------
 relational_operators/DestroyHashOperator.hpp    | 26 +++++++-------
 relational_operators/DropTableOperator.cpp      | 26 +++++++-------
 relational_operators/DropTableOperator.hpp      | 26 +++++++-------
 .../FinalizeAggregationOperator.cpp             | 26 +++++++-------
 .../FinalizeAggregationOperator.hpp             | 26 +++++++-------
 relational_operators/HashJoinOperator.cpp       | 28 +++++++--------
 relational_operators/HashJoinOperator.hpp       | 28 +++++++--------
 relational_operators/InsertOperator.cpp         | 26 +++++++-------
 relational_operators/InsertOperator.hpp         | 26 +++++++-------
 .../NestedLoopsJoinOperator.cpp                 | 26 +++++++-------
 .../NestedLoopsJoinOperator.hpp                 | 26 +++++++-------
 relational_operators/RebuildWorkOrder.hpp       | 26 +++++++-------
 relational_operators/RelationalOperator.hpp     | 26 +++++++-------
 .../RelationalOperatorsModule.hpp               | 26 +++++++-------
 relational_operators/SampleOperator.cpp         | 27 +++++++-------
 relational_operators/SampleOperator.hpp         | 26 +++++++-------
 relational_operators/SaveBlocksOperator.cpp     | 26 +++++++-------
 relational_operators/SaveBlocksOperator.hpp     | 26 +++++++-------
 relational_operators/SelectOperator.cpp         | 26 +++++++-------
 relational_operators/SelectOperator.hpp         | 26 +++++++-------
 relational_operators/SortMergeRunOperator.cpp   | 26 +++++++-------
 relational_operators/SortMergeRunOperator.hpp   | 26 +++++++-------
 relational_operators/SortMergeRunOperator.proto | 26 +++++++-------
 .../SortMergeRunOperatorHelpers.cpp             | 26 +++++++-------
 .../SortMergeRunOperatorHelpers.hpp             | 26 +++++++-------
 .../SortRunGenerationOperator.cpp               | 26 +++++++-------
 .../SortRunGenerationOperator.hpp               | 26 +++++++-------
 relational_operators/TableGeneratorOperator.cpp | 27 +++++++-------
 relational_operators/TableGeneratorOperator.hpp | 27 +++++++-------
 relational_operators/TextScanOperator.cpp       | 28 +++++++--------
 relational_operators/TextScanOperator.hpp       | 28 +++++++--------
 relational_operators/UpdateOperator.cpp         | 26 +++++++-------
 relational_operators/UpdateOperator.hpp         | 26 +++++++-------
 .../WindowAggregationOperator.cpp               | 28 +++++++--------
 .../WindowAggregationOperator.hpp               | 28 +++++++--------
 relational_operators/WorkOrder.hpp              | 26 +++++++-------
 relational_operators/WorkOrder.proto            | 28 +++++++--------
 relational_operators/WorkOrderFactory.cpp       | 25 +++++++------
 relational_operators/WorkOrderFactory.hpp       | 25 +++++++------
 .../tests/AggregationOperator_unittest.cpp      | 26 +++++++-------
 .../tests/HashJoinOperator_unittest.cpp         | 26 +++++++-------
 .../tests/SortMergeRunOperator_unittest.cpp     | 26 +++++++-------
 .../SortRunGenerationOperator_unittest.cpp      | 26 +++++++-------
 .../tests/TextScanOperator_unittest.cpp         | 25 +++++++------
 storage/AggregationOperationState.cpp           | 28 +++++++--------
 storage/AggregationOperationState.hpp           | 28 +++++++--------
 storage/AggregationOperationState.proto         | 28 +++++++--------
 .../BasicColumnStoreTupleStorageSubBlock.cpp    | 26 +++++++-------
 .../BasicColumnStoreTupleStorageSubBlock.hpp    | 26 +++++++-------
 storage/BasicColumnStoreValueAccessor.hpp       | 26 +++++++-------
 storage/BloomFilterIndexSubBlock.cpp            | 26 +++++++-------
 storage/BloomFilterIndexSubBlock.hpp            | 32 +++++++++--------
 storage/CMakeLists.txt                          | 28 +++++++--------
 storage/CSBTreeIndexSubBlock.cpp                | 28 +++++++--------
 storage/CSBTreeIndexSubBlock.hpp                | 28 +++++++--------
 storage/ColumnStoreUtil.cpp                     | 26 +++++++-------
 storage/ColumnStoreUtil.hpp                     | 26 +++++++-------
 storage/CompressedBlockBuilder.cpp              | 26 +++++++-------
 storage/CompressedBlockBuilder.hpp              | 26 +++++++-------
 ...ompressedColumnStoreTupleStorageSubBlock.cpp | 26 +++++++-------
 ...ompressedColumnStoreTupleStorageSubBlock.hpp | 26 +++++++-------
 storage/CompressedColumnStoreValueAccessor.hpp  | 26 +++++++-------
 ...ressedPackedRowStoreTupleStorageSubBlock.cpp | 26 +++++++-------
 ...ressedPackedRowStoreTupleStorageSubBlock.hpp | 26 +++++++-------
 .../CompressedPackedRowStoreValueAccessor.hpp   | 26 +++++++-------
 storage/CompressedStoreUtil.cpp                 | 26 +++++++-------
 storage/CompressedStoreUtil.hpp                 | 26 +++++++-------
 storage/CompressedTupleStorageSubBlock.cpp      | 26 +++++++-------
 storage/CompressedTupleStorageSubBlock.hpp      | 26 +++++++-------
 storage/CountedReference.hpp                    | 26 +++++++-------
 storage/DataExchange.proto                      | 25 +++++++------
 storage/DataExchangerAsync.cpp                  | 25 +++++++------
 storage/DataExchangerAsync.hpp                  | 25 +++++++------
 storage/EvictionPolicy.cpp                      | 32 +++++++++--------
 storage/EvictionPolicy.hpp                      | 26 +++++++-------
 storage/FileManager.hpp                         | 26 +++++++-------
 storage/FileManagerHdfs.cpp                     | 26 +++++++-------
 storage/FileManagerHdfs.hpp                     | 26 +++++++-------
 storage/FileManagerLocal.hpp                    | 25 +++++++------
 storage/FileManagerPosix.cpp                    | 26 +++++++-------
 storage/FileManagerPosix.hpp                    | 26 +++++++-------
 storage/FileManagerWindows.cpp                  | 26 +++++++-------
 storage/FileManagerWindows.hpp                  | 26 +++++++-------
 storage/HashTable.hpp                           | 28 +++++++--------
 storage/HashTable.proto                         | 28 +++++++--------
 storage/HashTableBase.hpp                       | 25 +++++++------
 storage/HashTableFactory.cpp                    | 25 +++++++------
 storage/HashTableFactory.hpp                    | 25 +++++++------
 storage/HashTableKeyManager.hpp                 | 27 +++++++-------
 storage/HashTablePool.hpp                       | 26 +++++++-------
 storage/IndexSubBlock.hpp                       | 26 +++++++-------
 storage/IndexSubBlockDescriptionFactory.hpp     | 26 +++++++-------
 storage/InsertDestination.cpp                   | 26 +++++++-------
 storage/InsertDestination.hpp                   | 26 +++++++-------
 storage/InsertDestination.proto                 | 26 +++++++-------
 storage/InsertDestinationInterface.hpp          | 25 +++++++------
 storage/LinearOpenAddressingHashTable.hpp       | 26 +++++++-------
 storage/PackedRowStoreTupleStorageSubBlock.cpp  | 28 +++++++--------
 storage/PackedRowStoreTupleStorageSubBlock.hpp  | 26 +++++++-------
 storage/PackedRowStoreValueAccessor.hpp         | 26 +++++++-------
 storage/PreloaderThread.cpp                     | 26 +++++++-------
 storage/PreloaderThread.hpp                     | 26 +++++++-------
 storage/SMAIndexSubBlock.cpp                    | 27 +++++++-------
 storage/SMAIndexSubBlock.hpp                    | 27 +++++++-------
 storage/SeparateChainingHashTable.hpp           | 26 +++++++-------
 .../SimpleScalarSeparateChainingHashTable.cpp   | 25 +++++++------
 .../SimpleScalarSeparateChainingHashTable.hpp   | 25 +++++++------
 storage/SplitRowStoreTupleStorageSubBlock.cpp   | 26 +++++++-------
 storage/SplitRowStoreTupleStorageSubBlock.hpp   | 26 +++++++-------
 storage/SplitRowStoreValueAccessor.hpp          | 26 +++++++-------
 storage/StorageBlob.hpp                         | 26 +++++++-------
 storage/StorageBlock.cpp                        | 28 +++++++--------
 storage/StorageBlock.hpp                        | 26 +++++++-------
 storage/StorageBlockBase.hpp                    | 26 +++++++-------
 storage/StorageBlockInfo.cpp                    | 26 +++++++-------
 storage/StorageBlockInfo.hpp                    | 26 +++++++-------
 storage/StorageBlockLayout.cpp                  | 28 +++++++--------
 storage/StorageBlockLayout.hpp                  | 26 +++++++-------
 storage/StorageBlockLayout.proto                | 30 ++++++++--------
 storage/StorageConfig.h.in                      | 26 +++++++-------
 storage/StorageConstants.hpp                    | 26 +++++++-------
 storage/StorageErrors.cpp                       | 26 +++++++-------
 storage/StorageErrors.hpp                       | 26 +++++++-------
 storage/StorageManager.cpp                      | 26 +++++++-------
 storage/StorageManager.hpp                      | 26 +++++++-------
 storage/StorageModule.hpp                       | 26 +++++++-------
 storage/SubBlockTypeRegistry.cpp                | 28 +++++++--------
 storage/SubBlockTypeRegistry.hpp                | 28 +++++++--------
 storage/SubBlockTypeRegistryMacros.hpp          | 26 +++++++-------
 storage/SubBlocksReference.hpp                  | 26 +++++++-------
 storage/TupleIdSequence.hpp                     | 26 +++++++-------
 storage/TupleReference.hpp                      | 26 +++++++-------
 storage/TupleStorageSubBlock.cpp                | 26 +++++++-------
 storage/TupleStorageSubBlock.hpp                | 26 +++++++-------
 storage/ValueAccessor.hpp                       | 26 +++++++-------
 storage/ValueAccessorUtil.hpp                   | 26 +++++++-------
 storage/WindowAggregationOperationState.cpp     | 28 +++++++--------
 storage/WindowAggregationOperationState.hpp     | 30 ++++++++--------
 storage/WindowAggregationOperationState.proto   | 28 +++++++--------
 .../AggregationOperationState_unittest.cpp      | 25 +++++++------
 ...ColumnStoreTupleStorageSubBlock_unittest.cpp | 26 +++++++-------
 .../tests/BloomFilterIndexSubBlock_unittest.cpp | 26 +++++++-------
 storage/tests/CSBTreeIndexSubBlock_unittest.cpp | 26 +++++++-------
 storage/tests/CSBTreePrettyPrinter.cpp          | 26 +++++++-------
 storage/tests/CSBTreePrettyPrinter.hpp          | 26 +++++++-------
 ...ColumnStoreTupleStorageSubBlock_unittest.cpp | 26 +++++++-------
 ...kedRowStoreTupleStorageSubBlock_unittest.cpp | 26 +++++++-------
 storage/tests/DataExchange_unittest.cpp         | 25 +++++++------
 storage/tests/EvictionPolicy_unittest.cpp       | 26 +++++++-------
 storage/tests/FileManagerHdfs_unittest.cpp      | 26 +++++++-------
 storage/tests/FileManagerLocal_unittest.cpp     | 26 +++++++-------
 storage/tests/FileManager_unittest_common.hpp   | 26 +++++++-------
 storage/tests/HashTable_unittest_common.hpp     | 26 +++++++-------
 .../LinearOpenAddressingHashTable_unittest.cpp  | 26 +++++++-------
 storage/tests/MockTupleStorageSubBlock.hpp      | 26 +++++++-------
 ...kedRowStoreTupleStorageSubBlock_unittest.cpp | 26 +++++++-------
 storage/tests/SMAIndexSubBlock_unittest.cpp     | 25 +++++++------
 .../SeparateChainingHashTable_unittest.cpp      | 26 +++++++-------
 ...ScalarSeparateChainingHashTable_unittest.cpp | 25 +++++++------
 ...litRowStoreTupleStorageSubBlock_unittest.cpp | 26 +++++++-------
 storage/tests/StorageBlockSort_unittest.cpp     | 26 +++++++-------
 storage/tests/StorageManager_unittest.cpp       | 25 +++++++------
 storage/tests/StorageTestConfig.h.in            | 25 +++++++------
 storage/tests/TupleStorePredicateUtil.hpp       | 26 +++++++-------
 ...WindowAggregationOperationState_unittest.cpp | 28 +++++++--------
 third_party/README.md                           |  4 +--
 third_party/glog/CMakeLists.txt                 | 21 ++++++++---
 third_party/protobuf_cmake/CMakeLists.txt       | 26 +++++++-------
 third_party/tmb/CMakeLists.txt                  | 26 +++++++-------
 third_party/tmb/benchmarks/CMakeLists.txt       | 23 ++++++------
 .../tmb/benchmarks/include/tmbbench/affinity.h  | 26 +++++++-------
 .../tmb/benchmarks/include/tmbbench/bus_setup.h | 26 +++++++-------
 .../tmb/benchmarks/include/tmbbench/messages.h  | 26 +++++++-------
 .../include/tmbbench/receiver_thread.h          | 26 +++++++-------
 .../benchmarks/include/tmbbench/sender_thread.h | 26 +++++++-------
 .../tmb/benchmarks/include/tmbbench/thread.h    | 26 +++++++-------
 third_party/tmb/benchmarks/src/affinity.cc      | 26 +++++++-------
 third_party/tmb/benchmarks/src/bus_setup.cc     | 26 +++++++-------
 .../tmb/benchmarks/src/oneway_throughput.cc     | 26 +++++++-------
 .../src/oneway_throughput_distributed.cc        | 26 +++++++-------
 ...oneway_throughput_distributed_coordinator.cc | 26 +++++++-------
 .../benchmarks/src/oneway_throughput_numa.cc    | 26 +++++++-------
 .../tmb/benchmarks/src/receiver_thread.cc       | 25 +++++++------
 third_party/tmb/benchmarks/src/reset_bus.cc     | 26 +++++++-------
 third_party/tmb/benchmarks/src/sender_thread.cc | 25 +++++++------
 third_party/tmb/benchmarks/src/thread.cc        | 25 +++++++------
 third_party/tmb/cmake/FindGrpc++.cmake          | 25 +++++++------
 third_party/tmb/cmake/FindLevelDB.cmake         | 25 +++++++------
 third_party/tmb/cmake/FindProtobuf3.cmake       | 25 +++++++------
 third_party/tmb/cmake/FindSQLite3.cmake         | 25 +++++++------
 third_party/tmb/cmake/FindVoltDB.cmake          | 25 +++++++------
 third_party/tmb/cmake/FindZookeeper.cmake       | 25 +++++++------
 third_party/tmb/include/tmb/address.h           | 31 +++++++++-------
 .../tmb/include/tmb/cancellation_token.h        | 31 +++++++++-------
 third_party/tmb/include/tmb/id_typedefs.h       | 31 +++++++++-------
 .../tmb/include/tmb/internal/c_string_buffer.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/cache_info.h       | 31 +++++++++-------
 .../tmb/include/tmb/internal/container_pusher.h | 31 +++++++++-------
 third_party/tmb/include/tmb/internal/crc32.h    | 31 +++++++++-------
 .../tmb/internal/heap_receiver_message_queue.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/iterator_adapter.h | 31 +++++++++-------
 .../tmb/internal/leveldb_key_comparator.h       | 31 +++++++++-------
 .../tmb/include/tmb/internal/leveldb_keys.h     | 31 +++++++++-------
 .../tmb/internal/lock_free_garbage_collector.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/lock_free_stack.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_read_status.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_reader_base.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_reader_posix.h | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_reader_stdio.h | 31 +++++++++-------
 .../include/tmb/internal/log_record_header.h    | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_writer_base.h  | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_writer_posix.h | 31 +++++++++-------
 .../tmb/include/tmb/internal/log_writer_stdio.h | 31 +++++++++-------
 .../include/tmb/internal/logging_constants.h    | 31 +++++++++-------
 .../tmb/internal/memory_based_message_bus.h     | 31 +++++++++-------
 .../internal/memory_mirror_cancellation_set.h   | 31 +++++++++-------
 .../tmb/internal/memory_mirror_delete_batch.h   | 31 +++++++++-------
 .../tmb/include/tmb/internal/message_metadata.h | 31 +++++++++-------
 .../tmb/internal/native_transaction_log.h       | 31 +++++++++-------
 .../tmb/internal/net_memory_container_pusher.h  | 31 +++++++++-------
 .../internal/net_message_removal_interface.h    | 31 +++++++++-------
 .../tmb/include/tmb/internal/net_service_impl.h | 31 +++++++++-------
 .../internal/persistent_bus_state_interface.h   | 31 +++++++++-------
 .../tmb/include/tmb/internal/queued_message.h   | 31 +++++++++-------
 third_party/tmb/include/tmb/internal/rcu.h      | 31 +++++++++-------
 .../tmb/include/tmb/internal/shared_bool.h      | 31 +++++++++-------
 .../include/tmb/internal/sqlite_connection.h    | 31 +++++++++-------
 .../tmb/internal/sqlite_connection_pool.h       | 31 +++++++++-------
 .../tmb/include/tmb/internal/threadsafe_set.h   | 31 +++++++++-------
 .../tmb/internal/tree_receiver_message_queue.h  | 31 +++++++++-------
 .../tmb/internal/voltdb_connection_pool.h       | 31 +++++++++-------
 .../tmb/internal/voltdb_procedure_factory.h     | 31 +++++++++-------
 .../tmb/internal/voltdb_procedure_warehouse.h   | 31 +++++++++-------
 .../tmb/include/tmb/internal/zookeeper_format.h | 31 +++++++++-------
 .../tmb/internal/zookeeper_receiver_context.h   | 31 +++++++++-------
 .../tmb/internal/zookeeper_transaction_batch.h  | 31 +++++++++-------
 .../tmb/include/tmb/leveldb_message_bus.h       | 31 +++++++++-------
 .../tmb/include/tmb/memory_mirror_message_bus.h | 31 +++++++++-------
 third_party/tmb/include/tmb/message_bus.h       | 31 +++++++++-------
 third_party/tmb/include/tmb/message_style.h     | 31 +++++++++-------
 .../include/tmb/native_logging_message_bus.h    | 31 +++++++++-------
 .../include/tmb/native_net_client_message_bus.h | 31 +++++++++-------
 third_party/tmb/include/tmb/priority.h          | 31 +++++++++-------
 .../tmb/include/tmb/pure_memory_message_bus.h   | 31 +++++++++-------
 .../tmb/include/tmb/sqlite_message_bus.h        | 31 +++++++++-------
 third_party/tmb/include/tmb/tagged_message.h    | 31 +++++++++-------
 .../tmb/include/tmb/voltdb_message_bus.h        | 31 +++++++++-------
 .../tmb/include/tmb/zookeeper_message_bus.h     | 31 +++++++++-------
 third_party/tmb/src/crc32.cc                    | 31 +++++++++-------
 .../tmb/src/heap_receiver_message_queue.cc      | 31 +++++++++-------
 third_party/tmb/src/java/CancelMessages.java    | 31 +++++++++-------
 third_party/tmb/src/java/ConnectClient.java     | 31 +++++++++-------
 third_party/tmb/src/java/DeleteMessages.java    | 31 +++++++++-------
 .../tmb/src/java/DeleteMessagesUnchecked.java   | 31 +++++++++-------
 third_party/tmb/src/java/DisconnectClient.java  | 31 +++++++++-------
 third_party/tmb/src/java/LoadState.java         | 31 +++++++++-------
 third_party/tmb/src/java/Receive.java           | 31 +++++++++-------
 third_party/tmb/src/java/ReceiveAndDelete.java  | 31 +++++++++-------
 third_party/tmb/src/java/RegisterReceiver.java  | 31 +++++++++-------
 third_party/tmb/src/java/RegisterSender.java    | 31 +++++++++-------
 third_party/tmb/src/java/ResetBus.java          | 31 +++++++++-------
 third_party/tmb/src/java/SendToAny.java         | 31 +++++++++-------
 .../tmb/src/java/SendToExplicitReceivers.java   | 31 +++++++++-------
 .../java/SendToExplicitReceiversUnchecked.java  | 31 +++++++++-------
 .../src/java/SendToSingleExplicitReceiver.java  | 31 +++++++++-------
 .../SendToSingleExplicitReceiverUnchecked.java  | 31 +++++++++-------
 third_party/tmb/src/leveldb_key_comparator.cc   | 31 +++++++++-------
 third_party/tmb/src/leveldb_message_bus.cc      | 31 +++++++++-------
 third_party/tmb/src/log_reader_posix.cc         | 31 +++++++++-------
 third_party/tmb/src/log_reader_stdio.cc         | 31 +++++++++-------
 third_party/tmb/src/log_writer_posix.cc         | 31 +++++++++-------
 third_party/tmb/src/log_writer_stdio.cc         | 31 +++++++++-------
 third_party/tmb/src/memory_based_message_bus.cc | 31 +++++++++-------
 .../tmb/src/memory_mirror_message_bus.cc        | 31 +++++++++-------
 third_party/tmb/src/message_bus.cc              | 31 +++++++++-------
 .../tmb/src/native_net_client_message_bus.cc    | 26 +++++++-------
 third_party/tmb/src/native_transaction_log.cc   | 31 +++++++++-------
 third_party/tmb/src/net_service_impl.cc         | 26 +++++++-------
 third_party/tmb/src/proto/tmb_net.proto         | 31 +++++++++-------
 third_party/tmb/src/pure_memory_message_bus.cc  | 31 +++++++++-------
 third_party/tmb/src/sql/voltdb_schema.sql       | 25 +++++++------
 third_party/tmb/src/sqlite_connection.cc        | 31 +++++++++-------
 third_party/tmb/src/sqlite_message_bus.cc       | 31 +++++++++-------
 third_party/tmb/src/tmb_net_server.cc           | 31 +++++++++-------
 .../tmb/src/tree_receiver_message_queue.cc      | 31 +++++++++-------
 third_party/tmb/src/voltdb_connection_pool.cc   | 31 +++++++++-------
 third_party/tmb/src/voltdb_message_bus.cc       | 31 +++++++++-------
 third_party/tmb/src/voltdb_procedure_factory.cc | 31 +++++++++-------
 third_party/tmb/src/zookeeper_message_bus.cc    | 31 +++++++++-------
 .../tmb/src/zookeeper_transaction_batch.cc      | 31 +++++++++-------
 .../tests/leveldb_message_bus_async_unittest.cc | 31 +++++++++-------
 .../tmb/tests/leveldb_message_bus_unittest.cc   | 31 +++++++++-------
 ..._mirror_message_bus_with_leveldb_unittest.cc | 31 +++++++++-------
 ...y_mirror_message_bus_with_sqlite_unittest.cc | 31 +++++++++-------
 ...y_mirror_message_bus_with_voltdb_unittest.cc | 31 +++++++++-------
 ...irror_message_bus_with_zookeeper_unittest.cc | 31 +++++++++-------
 .../tmb/tests/message_bus_unittest_common.h     | 31 +++++++++-------
 ...native_logging_message_bus_async_unittest.cc | 31 +++++++++-------
 .../native_logging_message_bus_unittest.cc      | 31 +++++++++-------
 .../native_net_client_message_bus_unittest.cc   | 31 +++++++++-------
 .../tests/pure_memory_message_bus_unittest.cc   | 31 +++++++++-------
 third_party/tmb/tests/rcu_unittest.cc           | 31 +++++++++-------
 .../tmb/tests/sqlite_message_bus_unittest.cc    | 31 +++++++++-------
 .../tmb/tests/voltdb_message_bus_unittest.cc    | 31 +++++++++-------
 .../tmb/tests/zookeeper_message_bus_unittest.cc | 31 +++++++++-------
 threading/CMakeLists.txt                        | 26 +++++++-------
 threading/ConditionVariable.cpp                 | 26 +++++++-------
 threading/ConditionVariable.hpp                 | 26 +++++++-------
 threading/Mutex.cpp                             | 26 +++++++-------
 threading/Mutex.hpp                             | 26 +++++++-------
 threading/SharedMutex.hpp                       | 26 +++++++-------
 threading/SpinMutex.hpp                         | 26 +++++++-------
 threading/SpinSharedMutex.hpp                   | 26 +++++++-------
 threading/Thread.cpp                            | 26 +++++++-------
 threading/Thread.hpp                            | 26 +++++++-------
 threading/ThreadIDBasedMap.hpp                  | 25 +++++++------
 threading/ThreadUtil.hpp                        | 26 +++++++-------
 threading/ThreadingConfig.h.in                  | 26 +++++++-------
 threading/ThreadingModule.hpp                   | 26 +++++++-------
 threading/WinThreadsAPI.hpp                     | 26 +++++++-------
 threading/cpp11/ConditionVariable.hpp           | 26 +++++++-------
 threading/cpp11/Mutex.hpp                       | 26 +++++++-------
 threading/cpp11/SharedMutex.hpp                 | 26 +++++++-------
 threading/cpp11/Thread.hpp                      | 26 +++++++-------
 threading/cpp11/cpp14/SharedMutex.hpp           | 26 +++++++-------
 threading/cpp11/cpp17/SharedMutex.hpp           | 25 +++++++------
 threading/posix/ConditionVariable.hpp           | 26 +++++++-------
 threading/posix/Mutex.hpp                       | 26 +++++++-------
 threading/posix/SharedMutex.hpp                 | 26 +++++++-------
 threading/posix/Thread.hpp                      | 26 +++++++-------
 threading/tests/Mutex_unittest.cpp              | 26 +++++++-------
 threading/tests/SharedMutex_unittest.cpp        | 26 +++++++-------
 threading/windows/ConditionVariable.hpp         | 26 +++++++-------
 threading/windows/Mutex.hpp                     | 26 +++++++-------
 threading/windows/SharedMutex.hpp               | 26 +++++++-------
 threading/windows/Thread.hpp                    | 26 +++++++-------
 transaction/AccessMode.cpp                      | 26 +++++++-------
 transaction/AccessMode.hpp                      | 26 +++++++-------
 transaction/CMakeLists.txt                      | 26 +++++++-------
 transaction/CycleDetector.cpp                   | 26 +++++++-------
 transaction/CycleDetector.hpp                   | 26 +++++++-------
 transaction/DeadLockDetector.cpp                | 26 +++++++-------
 transaction/DeadLockDetector.hpp                | 26 +++++++-------
 transaction/DirectedGraph.hpp                   | 26 +++++++-------
 transaction/Lock.hpp                            | 26 +++++++-------
 transaction/LockManager.cpp                     | 26 +++++++-------
 transaction/LockManager.hpp                     | 26 +++++++-------
 transaction/LockRequest.hpp                     | 26 +++++++-------
 transaction/LockTable.cpp                       | 26 +++++++-------
 transaction/LockTable.hpp                       | 26 +++++++-------
 transaction/ResourceId.cpp                      | 26 +++++++-------
 transaction/ResourceId.hpp                      | 26 +++++++-------
 transaction/StronglyConnectedComponents.cpp     | 26 +++++++-------
 transaction/StronglyConnectedComponents.hpp     | 26 +++++++-------
 transaction/Transaction.hpp                     | 26 +++++++-------
 transaction/TransactionModule.hpp               | 26 +++++++-------
 transaction/TransactionTable.cpp                | 26 +++++++-------
 transaction/TransactionTable.hpp                | 26 +++++++-------
 transaction/tests/AccessMode_unittest.cpp       | 26 +++++++-------
 transaction/tests/CycleDetector_unittest.cpp    | 26 +++++++-------
 transaction/tests/DeadLockDetector_unittest.cpp | 26 +++++++-------
 transaction/tests/DirectedGraph_unittest.cpp    | 26 +++++++-------
 transaction/tests/LockRequest_unittest.cpp      | 26 +++++++-------
 transaction/tests/LockTable_unittest.cpp        | 26 +++++++-------
 transaction/tests/Lock_unittest.cpp             | 26 +++++++-------
 transaction/tests/ResourceId_unittest.cpp       | 26 +++++++-------
 .../StronglyConnectedComponents_unittest.cpp    | 26 +++++++-------
 transaction/tests/TransactionTable_unittest.cpp | 26 +++++++-------
 types/CMakeLists.txt                            | 26 +++++++-------
 types/CharType.cpp                              | 26 +++++++-------
 types/CharType.hpp                              | 26 +++++++-------
 types/DateOperatorOverloads.hpp                 | 26 +++++++-------
 types/DateType.cpp                              | 23 +++++++-----
 types/DateType.hpp                              | 23 +++++++-----
 types/DatetimeIntervalType.cpp                  | 26 +++++++-------
 types/DatetimeIntervalType.hpp                  | 26 +++++++-------
 types/DatetimeLit.hpp                           | 26 +++++++-------
 types/DatetimeType.cpp                          | 26 +++++++-------
 types/DatetimeType.hpp                          | 26 +++++++-------
 types/DoubleType.cpp                            | 26 +++++++-------
 types/DoubleType.hpp                            | 26 +++++++-------
 types/FloatType.cpp                             | 26 +++++++-------
 types/FloatType.hpp                             | 26 +++++++-------
 types/IntType.cpp                               | 26 +++++++-------
 types/IntType.hpp                               | 26 +++++++-------
 types/IntervalLit.hpp                           | 26 +++++++-------
 types/IntervalParser.cpp                        | 26 +++++++-------
 types/IntervalParser.hpp                        | 26 +++++++-------
 types/LongType.cpp                              | 26 +++++++-------
 types/LongType.hpp                              | 26 +++++++-------
 types/NullCoercibilityCheckMacro.hpp            | 25 +++++++------
 types/NullType.hpp                              | 26 +++++++-------
 types/NumericSuperType.hpp                      | 26 +++++++-------
 types/NumericTypeUnifier.hpp                    | 25 +++++++------
 types/Type.cpp                                  | 26 +++++++-------
 types/Type.hpp                                  | 26 +++++++-------
 types/Type.proto                                | 26 +++++++-------
 types/TypeErrors.hpp                            | 26 +++++++-------
 types/TypeFactory.cpp                           | 26 +++++++-------
 types/TypeFactory.hpp                           | 26 +++++++-------
 types/TypeID.cpp                                | 26 +++++++-------
 types/TypeID.hpp                                | 26 +++++++-------
 types/TypedValue.cpp                            | 26 +++++++-------
 types/TypedValue.hpp                            | 26 +++++++-------
 types/TypedValue.proto                          | 26 +++++++-------
 types/TypesModule.hpp                           | 26 +++++++-------
 types/VarCharType.cpp                           | 26 +++++++-------
 types/VarCharType.hpp                           | 26 +++++++-------
 types/YearMonthIntervalType.cpp                 | 26 +++++++-------
 types/YearMonthIntervalType.hpp                 | 26 +++++++-------
 types/containers/CMakeLists.txt                 | 26 +++++++-------
 types/containers/ColumnVector.cpp               | 26 +++++++-------
 types/containers/ColumnVector.hpp               | 26 +++++++-------
 types/containers/ColumnVectorUtil.hpp           | 26 +++++++-------
 types/containers/ColumnVectorsValueAccessor.hpp | 26 +++++++-------
 types/containers/Tuple.hpp                      | 26 +++++++-------
 types/containers/Tuple.proto                    | 25 +++++++------
 .../containers/tests/ColumnVector_unittest.cpp  | 26 +++++++-------
 types/operations/CMakeLists.txt                 | 26 +++++++-------
 types/operations/Operation.cpp                  | 26 +++++++-------
 types/operations/Operation.hpp                  | 26 +++++++-------
 types/operations/Operation.proto                | 30 ++++++++--------
 .../binary_operations/AddBinaryOperation.cpp    | 28 +++++++--------
 .../binary_operations/AddBinaryOperation.hpp    | 26 +++++++-------
 .../ArithmeticBinaryOperation.hpp               | 28 +++++++--------
 .../ArithmeticBinaryOperators.hpp               | 28 +++++++--------
 .../binary_operations/BinaryOperation.cpp       | 28 +++++++--------
 .../binary_operations/BinaryOperation.hpp       | 26 +++++++-------
 .../BinaryOperationFactory.cpp                  | 28 +++++++--------
 .../BinaryOperationFactory.hpp                  | 26 +++++++-------
 .../binary_operations/BinaryOperationID.cpp     | 28 +++++++--------
 .../binary_operations/BinaryOperationID.hpp     | 28 +++++++--------
 .../operations/binary_operations/CMakeLists.txt | 28 +++++++--------
 .../binary_operations/DivideBinaryOperation.cpp | 28 +++++++--------
 .../binary_operations/DivideBinaryOperation.hpp | 26 +++++++-------
 .../binary_operations/ModuloBinaryOperation.cpp | 27 +++++++-------
 .../binary_operations/ModuloBinaryOperation.hpp | 27 +++++++-------
 .../MultiplyBinaryOperation.cpp                 | 28 +++++++--------
 .../MultiplyBinaryOperation.hpp                 | 26 +++++++-------
 .../SubtractBinaryOperation.cpp                 | 28 +++++++--------
 .../SubtractBinaryOperation.hpp                 | 26 +++++++-------
 .../tests/AddBinaryOperation_unittest.cpp       | 25 +++++++------
 .../tests/BinaryOperationTestUtil.hpp           | 25 +++++++------
 .../tests/BinaryOperation_unittest.cpp          | 28 +++++++--------
 .../tests/DivideBinaryOperation_unittest.cpp    | 25 +++++++------
 .../tests/ModuloBinaryOperation_unittest.cpp    | 27 +++++++-------
 .../tests/MultiplyBinaryOperation_unittest.cpp  | 25 +++++++------
 .../tests/SubtractBinaryOperation_unittest.cpp  | 25 +++++++------
 .../comparisons/AsciiStringComparators-inl.hpp  | 26 +++++++-------
 .../comparisons/AsciiStringComparators.hpp      | 26 +++++++-------
 .../operations/comparisons/BasicComparison.cpp  | 26 +++++++-------
 .../operations/comparisons/BasicComparison.hpp  | 26 +++++++-------
 types/operations/comparisons/CMakeLists.txt     | 26 +++++++-------
 types/operations/comparisons/Comparison-inl.hpp | 26 +++++++-------
 types/operations/comparisons/Comparison.cpp     | 26 +++++++-------
 types/operations/comparisons/Comparison.hpp     | 26 +++++++-------
 .../comparisons/ComparisonFactory.cpp           | 26 +++++++-------
 .../comparisons/ComparisonFactory.hpp           | 26 +++++++-------
 types/operations/comparisons/ComparisonID.cpp   | 28 +++++++--------
 types/operations/comparisons/ComparisonID.hpp   | 32 +++++++++--------
 types/operations/comparisons/ComparisonUtil.hpp | 26 +++++++-------
 .../operations/comparisons/EqualComparison.cpp  | 26 +++++++-------
 .../operations/comparisons/EqualComparison.hpp  | 26 +++++++-------
 .../comparisons/GreaterComparison.cpp           | 26 +++++++-------
 .../comparisons/GreaterComparison.hpp           | 26 +++++++-------
 .../comparisons/GreaterOrEqualComparison.cpp    | 26 +++++++-------
 .../comparisons/GreaterOrEqualComparison.hpp    | 26 +++++++-------
 types/operations/comparisons/LessComparison.cpp | 26 +++++++-------
 types/operations/comparisons/LessComparison.hpp | 26 +++++++-------
 .../comparisons/LessOrEqualComparison.cpp       | 26 +++++++-------
 .../comparisons/LessOrEqualComparison.hpp       | 26 +++++++-------
 .../comparisons/LiteralComparators-inl.hpp      | 26 +++++++-------
 .../comparisons/LiteralComparators.hpp          | 26 +++++++-------
 .../comparisons/NotEqualComparison.cpp          | 26 +++++++-------
 .../comparisons/NotEqualComparison.hpp          | 26 +++++++-------
 .../PatternMatchingComparators-inl.hpp          | 26 +++++++-------
 .../comparisons/PatternMatchingComparators.hpp  | 26 +++++++-------
 .../comparisons/PatternMatchingComparison.cpp   | 26 +++++++-------
 .../comparisons/PatternMatchingComparison.hpp   | 26 +++++++-------
 .../comparisons/tests/Comparison_unittest.cpp   | 26 +++++++-------
 .../ArithmeticUnaryOperations.cpp               | 26 +++++++-------
 .../ArithmeticUnaryOperations.hpp               | 26 +++++++-------
 .../ArithmeticUnaryOperators.hpp                | 26 +++++++-------
 .../operations/unary_operations/CMakeLists.txt  | 28 +++++++--------
 .../unary_operations/DateExtractOperation.cpp   | 26 +++++++-------
 .../unary_operations/DateExtractOperation.hpp   | 26 +++++++-------
 .../unary_operations/NumericCastOperation.hpp   | 26 +++++++-------
 .../unary_operations/SubstringOperation.cpp     | 26 +++++++-------
 .../unary_operations/SubstringOperation.hpp     | 26 +++++++-------
 .../unary_operations/UnaryOperation.cpp         | 28 +++++++--------
 .../unary_operations/UnaryOperation.hpp         | 26 +++++++-------
 .../unary_operations/UnaryOperationFactory.cpp  | 28 +++++++--------
 .../unary_operations/UnaryOperationFactory.hpp  | 26 +++++++-------
 .../unary_operations/UnaryOperationID.cpp       | 28 +++++++--------
 .../unary_operations/UnaryOperationID.hpp       | 28 +++++++--------
 .../tests/DateExtractOperation_unittest.cpp     | 26 +++++++-------
 .../tests/NegateUnaryOperation_unittest.cpp     | 25 +++++++------
 .../tests/NumericCastOperation_unittest.cpp     | 25 +++++++------
 .../tests/UnaryOperation_unittest.cpp           | 26 +++++++-------
 types/port/CMakeLists.txt                       | 26 +++++++-------
 types/port/TypesPortConfig.h.in                 | 26 +++++++-------
 types/port/gmtime_r.hpp                         | 26 +++++++-------
 types/port/localtime_r.hpp                      | 26 +++++++-------
 types/port/strnlen.hpp                          | 26 +++++++-------
 types/port/tests/timegm_benchmark.cpp           | 25 +++++++------
 types/port/tests/timegm_unittest.cpp            | 25 +++++++------
 types/port/timegm.cpp                           | 26 +++++++-------
 types/port/timegm.hpp                           | 30 ++++++++--------
 types/tests/CharType_unittest.cpp               | 25 +++++++------
 types/tests/DateType_unittest.cpp               | 23 +++++++-----
 types/tests/DatetimeIntervalType_unittest.cpp   | 25 +++++++------
 types/tests/DatetimeType_unittest.cpp           | 25 +++++++------
 types/tests/DoubleType_unittest.cpp             | 25 +++++++------
 types/tests/FloatType_unittest.cpp              | 25 +++++++------
 types/tests/IntType_unittest.cpp                | 25 +++++++------
 types/tests/LongType_unittest.cpp               | 25 +++++++------
 types/tests/TypeTest_common.hpp                 | 25 +++++++------
 types/tests/Type_unittest.cpp                   | 26 +++++++-------
 types/tests/TypedValue_unittest.cpp             | 26 +++++++-------
 types/tests/VarCharType_unittest.cpp            | 25 +++++++------
 types/tests/YearMonthIntervalType_unittest.cpp  | 25 +++++++------
 utility/Alignment.hpp                           | 26 +++++++-------
 utility/BitManipulation.hpp                     | 26 +++++++-------
 utility/BitVector.hpp                           | 26 +++++++-------
 utility/BloomFilter.hpp                         | 37 +++++++++----------
 utility/BloomFilter.proto                       | 26 +++++++-------
 utility/CMakeLists.txt                          | 26 +++++++-------
 utility/CalculateInstalledMemory.cpp            | 25 +++++++------
 utility/CalculateInstalledMemory.hpp            | 25 +++++++------
 utility/Cast.hpp                                | 26 +++++++-------
 utility/CheckSnprintf.hpp                       | 26 +++++++-------
 utility/DAG.hpp                                 | 26 +++++++-------
 utility/EqualsAnyConstant.hpp                   | 25 +++++++------
 utility/ExecutionDAGVisualizer.cpp              | 26 +++++++-------
 utility/ExecutionDAGVisualizer.hpp              | 26 +++++++-------
 utility/Glob.cpp                                | 26 +++++++-------
 utility/Glob.hpp                                | 26 +++++++-------
 utility/HashPair.hpp                            | 26 +++++++-------
 utility/Macros.hpp                              | 26 +++++++-------
 utility/MemStream.hpp                           | 25 +++++++------
 utility/PlanVisualizer.cpp                      | 26 +++++++-------
 utility/PlanVisualizer.hpp                      | 26 +++++++-------
 utility/PrimeNumber.cpp                         | 26 +++++++-------
 utility/PrimeNumber.hpp                         | 26 +++++++-------
 utility/PtrList.hpp                             | 26 +++++++-------
 utility/PtrMap.hpp                              | 26 +++++++-------
 utility/PtrVector.hpp                           | 26 +++++++-------
 utility/ScopedBuffer.hpp                        | 26 +++++++-------
 utility/ScopedDeleter.hpp                       | 26 +++++++-------
 utility/ShardedLockManager.hpp                  | 26 +++++++-------
 utility/SortConfiguration.cpp                   | 25 +++++++------
 utility/SortConfiguration.hpp                   | 26 +++++++-------
 utility/SortConfiguration.proto                 | 25 +++++++------
 utility/SqlError.cpp                            | 26 +++++++-------
 utility/SqlError.hpp                            | 26 +++++++-------
 utility/StringUtil.cpp                          | 26 +++++++-------
 utility/StringUtil.hpp                          | 28 +++++++--------
 utility/TemplateUtil.hpp                        | 26 +++++++-------
 utility/ThreadSafeQueue.hpp                     | 26 +++++++-------
 utility/TreeStringSerializable.hpp              | 26 +++++++-------
 utility/UtilityConfig.h.in                      | 26 +++++++-------
 utility/UtilityModule.hpp                       | 26 +++++++-------
 utility/VectorUtil.hpp                          | 26 +++++++-------
 utility/tests/BitVector_unittest.cpp            | 26 +++++++-------
 utility/tests/BloomFilter_unittest.cpp          | 26 +++++++-------
 .../tests/CalculateInstalledMemory_unittest.cpp | 25 +++++++------
 utility/tests/DAG_unittest.cpp                  | 26 +++++++-------
 utility/tests/EqualsAnyConstant_benchmark.cpp   | 26 +++++++-------
 utility/tests/EqualsAnyConstant_unittest.cpp    | 25 +++++++------
 utility/tests/HashPair_benchmark.cpp            | 26 +++++++-------
 utility/tests/PrimeNumber_unittest.cpp          | 26 +++++++-------
 utility/tests/ScopedDeleter_unittest.cpp        | 26 +++++++-------
 utility/tests/SqlError_unittest.cpp             | 26 +++++++-------
 utility/tests/TemplateUtil_unittest.cpp         | 26 +++++++-------
 utility/tests/TextBasedTestDriver_unittest.cpp  | 26 +++++++-------
 utility/tests/ThreadSafeQueue_unittest.cpp      | 26 +++++++-------
 .../tests/TreeStringSerializable_unittest.cpp   | 26 +++++++-------
 utility/textbased_test/TextBasedTest.cpp        | 26 +++++++-------
 utility/textbased_test/TextBasedTest.hpp        | 26 +++++++-------
 utility/textbased_test/TextBasedTestDriver.cpp  | 26 +++++++-------
 utility/textbased_test/TextBasedTestDriver.hpp  | 26 +++++++-------
 utility/textbased_test/TextBasedTestRunner.hpp  | 26 +++++++-------
 validate_cmakelists.py                          | 26 +++++++-------
 yarn/CMakeLists.txt                             | 26 +++++++-------
 1175 files changed, 16939 insertions(+), 14427 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 3192713..487aaf9 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 cmake_minimum_required (VERSION 2.8.6)
 project (QUICKSTEP)
@@ -281,13 +283,11 @@ else()
     set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror")
   endif()
 
-  if (ENABLE_DISTRIBUTED)
-    # Clang reports such warning when using Protoc 3.0 beta.
-    if(${CMAKE_CXX_COMPILER_ID} MATCHES "Clang")
-      CHECK_CXX_COMPILER_FLAG("-Wno-extended-offsetof" COMPILER_HAS_WNO_EXTENDED_OFFSETOF)
-      if (COMPILER_HAS_WNO_EXTENDED_OFFSETOF)
-        set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-extended-offsetof")
-      endif()
+  # Clang reports such warning when using Protoc 3.0 beta.
+  if(${CMAKE_CXX_COMPILER_ID} MATCHES "Clang")
+    CHECK_CXX_COMPILER_FLAG("-Wno-extended-offsetof" COMPILER_HAS_WNO_EXTENDED_OFFSETOF)
+    if (COMPILER_HAS_WNO_EXTENDED_OFFSETOF)
+      set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-extended-offsetof")
     endif()
   endif()
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/CREDITS.md
----------------------------------------------------------------------
diff --git a/CREDITS.md b/CREDITS.md
index 5b6324c..353456a 100644
--- a/CREDITS.md
+++ b/CREDITS.md
@@ -1,4 +1,4 @@
-At the time of the intial commit in January 2016, here are the commit stats of the Quickstep committers over the past many years. 
+At the time of the intial commit in January 2016, here are the commit stats of the Quickstep committers over the past many years.
 
 | **Committer**   | **# commits**    | **++ lines**      | **-- lines**      |
 | :-------------- |-----------------:|------------------:|------------------:|

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/GENERATOR_FUNCTIONS.md
----------------------------------------------------------------------
diff --git a/GENERATOR_FUNCTIONS.md b/GENERATOR_FUNCTIONS.md
index 89f6909..78b4079 100644
--- a/GENERATOR_FUNCTIONS.md
+++ b/GENERATOR_FUNCTIONS.md
@@ -65,12 +65,12 @@ It is a theta-join between two generated relations. The output will be:
 
 4. At the final stage of query planning, the physical plan node is transformed into a relational operator.
  - *See `TableGeneratorOperator` as the relational operator class that represents a operator that will produce run-time work orders.*
- 
+
 5. At the time of query execution, the function handle's `populateColumns()` method is invoked to actually generate a relation.
  - *See `TableGeneratorWorkOrder` as the work order class that actually invokes the generator function to populate a relation at run time.*
- 
+
 ##Implement a new generator function
-There are just three steps to implement a new generator function. 
+There are just three steps to implement a new generator function.
 
 1. Subclass **`GeneratorFunctionHandle`** and implement all virtual methods.
  - *See `GenerateSeriesHandle` as an example.*

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 32db8b8..3ead237 100644
--- a/NOTICE
+++ b/NOTICE
@@ -25,7 +25,7 @@ Portions Copyright (c) Shuo Chen <ch...@chenshuo.com>
 Portions Copyright (c) Yusuke Suzuki <ut...@gmail.com>
 
 [Copyright for third_party/cpplint]
-Portions Copyright (c) 2009 Google Inc 
+Portions Copyright (c) 2009 Google Inc
 
 [Copyright for third_party/farmhash]
 Copyright (c) 2014 Google, Inc.


[20/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/StorageTestConfig.h.in
----------------------------------------------------------------------
diff --git a/storage/tests/StorageTestConfig.h.in b/storage/tests/StorageTestConfig.h.in
index 9db581d..4ffb09a 100644
--- a/storage/tests/StorageTestConfig.h.in
+++ b/storage/tests/StorageTestConfig.h.in
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_LONG_HASH_REVERSIBLE

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/TupleStorePredicateUtil.hpp
----------------------------------------------------------------------
diff --git a/storage/tests/TupleStorePredicateUtil.hpp b/storage/tests/TupleStorePredicateUtil.hpp
index a2a5777..dcdf0fa 100644
--- a/storage/tests/TupleStorePredicateUtil.hpp
+++ b/storage/tests/TupleStorePredicateUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_TESTS_TUPLE_STORE_PREDICATE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/tests/WindowAggregationOperationState_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/WindowAggregationOperationState_unittest.cpp b/storage/tests/WindowAggregationOperationState_unittest.cpp
index d58f0f5..c12f66f 100644
--- a/storage/tests/WindowAggregationOperationState_unittest.cpp
+++ b/storage/tests/WindowAggregationOperationState_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/README.md
----------------------------------------------------------------------
diff --git a/third_party/README.md b/third_party/README.md
index 3109ca1..cfb1a51 100644
--- a/third_party/README.md
+++ b/third_party/README.md
@@ -2,6 +2,6 @@
 
 This directory includes various open-source third-party code that is used by
 Quickstep. Some code has been modified slightly to fix build issues or to integrate
-with Quickstep. With the exception of the code in the `tmb` and the `protobuf_cmake` 
-directories (which are part of the Quickstep project itself), all libraries here 
+with Quickstep. With the exception of the code in the `tmb` and the `protobuf_cmake`
+directories (which are part of the Quickstep project itself), all libraries here
 belong to their original authors and are governed by their respective licenses.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/glog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/third_party/glog/CMakeLists.txt b/third_party/glog/CMakeLists.txt
index afd9e28..a916b96 100644
--- a/third_party/glog/CMakeLists.txt
+++ b/third_party/glog/CMakeLists.txt
@@ -1,6 +1,19 @@
-# This file copyright (c) 2011-2015, Quickstep Technologies LLC.
-# All rights reserved.
-# See file CREDITS.txt for details.
+# 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.
 
 # This file allows building glog with CMake instead of autotools. glog itself
 # is copyright Google, Inc. and is covered by the license described in the
@@ -131,7 +144,7 @@ else()
     set(UCONTEXT_INCLUDES "${UCONTEXT_INCLUDES}\n#include <ucontext.h>")
   endif()
 
-  foreach (pc_field          
+  foreach (pc_field
            "uc_mcontext.gregs[REG_EIP]"
            "uc_mcontext.gregs[REG_RIP]"
            "uc_mcontext.sc_ip"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/protobuf_cmake/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/third_party/protobuf_cmake/CMakeLists.txt b/third_party/protobuf_cmake/CMakeLists.txt
index 22c6194..7e08056 100644
--- a/third_party/protobuf_cmake/CMakeLists.txt
+++ b/third_party/protobuf_cmake/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # This CMakeLists file allows building protobufs as part of the regular CMake
 # build process instead of using autotools (on UNIX) or Visual Studio projects

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/third_party/tmb/CMakeLists.txt b/third_party/tmb/CMakeLists.txt
index f22efa7..14e467b 100644
--- a/third_party/tmb/CMakeLists.txt
+++ b/third_party/tmb/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 cmake_minimum_required(VERSION 2.8)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/CMakeLists.txt b/third_party/tmb/benchmarks/CMakeLists.txt
index ca08a24..cec8fd8 100644
--- a/third_party/tmb/benchmarks/CMakeLists.txt
+++ b/third_party/tmb/benchmarks/CMakeLists.txt
@@ -1,17 +1,14 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, 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
 #
-#   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
 #
-#       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.
+# 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.
 
 find_package(Threads REQUIRED)
 set(CMAKE_REQUIRED_LIBRARIES ${CMAKE_REQUIRED_LIBRARIES} ${CMAKE_THREAD_LIBS_INIT})
@@ -88,7 +85,7 @@ endif()
 if (TMB_AFFINITY_LINUX)
   set_property(
       DIRECTORY
-      APPEND PROPERTY COMPILE_DEFINITIONS TMB_AFFINITY_LINUX) 
+      APPEND PROPERTY COMPILE_DEFINITIONS TMB_AFFINITY_LINUX)
 elseif (TMB_AFFINITY_FREEBSD)
   set_property(
       DIRECTORY

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/include/tmbbench/affinity.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/include/tmbbench/affinity.h b/third_party/tmb/benchmarks/include/tmbbench/affinity.h
index 85fa1ea..bc09000 100644
--- a/third_party/tmb/benchmarks/include/tmbbench/affinity.h
+++ b/third_party/tmb/benchmarks/include/tmbbench/affinity.h
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 TMBBENCH_AFFINITY_H_
 #define TMBBENCH_AFFINITY_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/include/tmbbench/bus_setup.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/include/tmbbench/bus_setup.h b/third_party/tmb/benchmarks/include/tmbbench/bus_setup.h
index b4ac383..2e06351 100644
--- a/third_party/tmb/benchmarks/include/tmbbench/bus_setup.h
+++ b/third_party/tmb/benchmarks/include/tmbbench/bus_setup.h
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 TMBBENCH_BUS_SETUP_H_
 #define TMBBENCH_BUS_SETUP_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/include/tmbbench/messages.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/include/tmbbench/messages.h b/third_party/tmb/benchmarks/include/tmbbench/messages.h
index ce9fadb..5527f7f 100644
--- a/third_party/tmb/benchmarks/include/tmbbench/messages.h
+++ b/third_party/tmb/benchmarks/include/tmbbench/messages.h
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 TMBBENCH_MESSAGES_H_
 #define TMBBENCH_MESSAGES_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/include/tmbbench/receiver_thread.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/include/tmbbench/receiver_thread.h b/third_party/tmb/benchmarks/include/tmbbench/receiver_thread.h
index 561c4f2..f09bc75 100644
--- a/third_party/tmb/benchmarks/include/tmbbench/receiver_thread.h
+++ b/third_party/tmb/benchmarks/include/tmbbench/receiver_thread.h
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 TMBBENCH_RECEIVER_THREAD_H_
 #define TMBBENCH_RECEIVER_THREAD_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/include/tmbbench/sender_thread.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/include/tmbbench/sender_thread.h b/third_party/tmb/benchmarks/include/tmbbench/sender_thread.h
index 14d0e8e..495a087 100644
--- a/third_party/tmb/benchmarks/include/tmbbench/sender_thread.h
+++ b/third_party/tmb/benchmarks/include/tmbbench/sender_thread.h
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 TMBBENCH_SENDER_THREAD_H_
 #define TMBBENCH_SENDER_THREAD_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/include/tmbbench/thread.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/include/tmbbench/thread.h b/third_party/tmb/benchmarks/include/tmbbench/thread.h
index 1f5b672..9eae6fb 100644
--- a/third_party/tmb/benchmarks/include/tmbbench/thread.h
+++ b/third_party/tmb/benchmarks/include/tmbbench/thread.h
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 TMBBENCH_THREAD_H_
 #define TMBBENCH_THREAD_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/affinity.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/affinity.cc b/third_party/tmb/benchmarks/src/affinity.cc
index 9c3b6ea..0cf9344 100644
--- a/third_party/tmb/benchmarks/src/affinity.cc
+++ b/third_party/tmb/benchmarks/src/affinity.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 "tmbbench/affinity.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/bus_setup.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/bus_setup.cc b/third_party/tmb/benchmarks/src/bus_setup.cc
index 3fc9c9c..8e4c223 100644
--- a/third_party/tmb/benchmarks/src/bus_setup.cc
+++ b/third_party/tmb/benchmarks/src/bus_setup.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 "tmbbench/bus_setup.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/oneway_throughput.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/oneway_throughput.cc b/third_party/tmb/benchmarks/src/oneway_throughput.cc
index 54c78a7..f123fb3 100644
--- a/third_party/tmb/benchmarks/src/oneway_throughput.cc
+++ b/third_party/tmb/benchmarks/src/oneway_throughput.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 <chrono>  // NOLINT(build/c++11)
 #include <iostream>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/oneway_throughput_distributed.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/oneway_throughput_distributed.cc b/third_party/tmb/benchmarks/src/oneway_throughput_distributed.cc
index bd581e3..08e2b9f 100644
--- a/third_party/tmb/benchmarks/src/oneway_throughput_distributed.cc
+++ b/third_party/tmb/benchmarks/src/oneway_throughput_distributed.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 <memory>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/oneway_throughput_distributed_coordinator.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/oneway_throughput_distributed_coordinator.cc b/third_party/tmb/benchmarks/src/oneway_throughput_distributed_coordinator.cc
index 5ac2702..a5e15f7 100644
--- a/third_party/tmb/benchmarks/src/oneway_throughput_distributed_coordinator.cc
+++ b/third_party/tmb/benchmarks/src/oneway_throughput_distributed_coordinator.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 <chrono>  // NOLINT(build/c++11)
 #include <iostream>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/oneway_throughput_numa.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/oneway_throughput_numa.cc b/third_party/tmb/benchmarks/src/oneway_throughput_numa.cc
index 33084f2..4d228b4 100644
--- a/third_party/tmb/benchmarks/src/oneway_throughput_numa.cc
+++ b/third_party/tmb/benchmarks/src/oneway_throughput_numa.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 <chrono>  // NOLINT(build/c++11)
 #include <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/receiver_thread.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/receiver_thread.cc b/third_party/tmb/benchmarks/src/receiver_thread.cc
index 8420b91..3aac95b 100644
--- a/third_party/tmb/benchmarks/src/receiver_thread.cc
+++ b/third_party/tmb/benchmarks/src/receiver_thread.cc
@@ -1,16 +1,19 @@
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 "tmbbench/receiver_thread.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/reset_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/reset_bus.cc b/third_party/tmb/benchmarks/src/reset_bus.cc
index 2ac90c5..9c2310e 100644
--- a/third_party/tmb/benchmarks/src/reset_bus.cc
+++ b/third_party/tmb/benchmarks/src/reset_bus.cc
@@ -1,17 +1,19 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 <memory>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/sender_thread.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/sender_thread.cc b/third_party/tmb/benchmarks/src/sender_thread.cc
index ad207cb..0cf41f6 100644
--- a/third_party/tmb/benchmarks/src/sender_thread.cc
+++ b/third_party/tmb/benchmarks/src/sender_thread.cc
@@ -1,16 +1,19 @@
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 "tmbbench/sender_thread.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/benchmarks/src/thread.cc
----------------------------------------------------------------------
diff --git a/third_party/tmb/benchmarks/src/thread.cc b/third_party/tmb/benchmarks/src/thread.cc
index e1711d8..b2290b3 100644
--- a/third_party/tmb/benchmarks/src/thread.cc
+++ b/third_party/tmb/benchmarks/src/thread.cc
@@ -1,16 +1,19 @@
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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 "tmbbench/thread.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/cmake/FindGrpc++.cmake
----------------------------------------------------------------------
diff --git a/third_party/tmb/cmake/FindGrpc++.cmake b/third_party/tmb/cmake/FindGrpc++.cmake
index c3d7196..371d870 100644
--- a/third_party/tmb/cmake/FindGrpc++.cmake
+++ b/third_party/tmb/cmake/FindGrpc++.cmake
@@ -1,16 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find grpc core library and c++ bindings (grpc++).
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/cmake/FindLevelDB.cmake
----------------------------------------------------------------------
diff --git a/third_party/tmb/cmake/FindLevelDB.cmake b/third_party/tmb/cmake/FindLevelDB.cmake
index b4d5b0f..0b0a761 100644
--- a/third_party/tmb/cmake/FindLevelDB.cmake
+++ b/third_party/tmb/cmake/FindLevelDB.cmake
@@ -1,16 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find LevelDB.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/cmake/FindProtobuf3.cmake
----------------------------------------------------------------------
diff --git a/third_party/tmb/cmake/FindProtobuf3.cmake b/third_party/tmb/cmake/FindProtobuf3.cmake
index b5c8084..9a5ad4a 100644
--- a/third_party/tmb/cmake/FindProtobuf3.cmake
+++ b/third_party/tmb/cmake/FindProtobuf3.cmake
@@ -1,16 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find Protocol Buffers supporting proto3 syntax.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/cmake/FindSQLite3.cmake
----------------------------------------------------------------------
diff --git a/third_party/tmb/cmake/FindSQLite3.cmake b/third_party/tmb/cmake/FindSQLite3.cmake
index fa8de86..98e78c6 100644
--- a/third_party/tmb/cmake/FindSQLite3.cmake
+++ b/third_party/tmb/cmake/FindSQLite3.cmake
@@ -1,16 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find SQLite 3.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/cmake/FindVoltDB.cmake
----------------------------------------------------------------------
diff --git a/third_party/tmb/cmake/FindVoltDB.cmake b/third_party/tmb/cmake/FindVoltDB.cmake
index da39f7c..0b3c5a6 100644
--- a/third_party/tmb/cmake/FindVoltDB.cmake
+++ b/third_party/tmb/cmake/FindVoltDB.cmake
@@ -1,16 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find VoltDB C++ client library.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/cmake/FindZookeeper.cmake
----------------------------------------------------------------------
diff --git a/third_party/tmb/cmake/FindZookeeper.cmake b/third_party/tmb/cmake/FindZookeeper.cmake
index ab20405..e8d1877 100644
--- a/third_party/tmb/cmake/FindZookeeper.cmake
+++ b/third_party/tmb/cmake/FindZookeeper.cmake
@@ -1,16 +1,19 @@
-#   Copyright 2014-2015 Quickstep Technologies LLC.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Module to find Zookeeper C Binding.
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/address.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/address.h b/third_party/tmb/include/tmb/address.h
index f09e765..e7ed3be 100644
--- a/third_party/tmb/include/tmb/address.h
+++ b/third_party/tmb/include/tmb/address.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_ADDRESS_H_
 #define TMB_ADDRESS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/cancellation_token.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/cancellation_token.h b/third_party/tmb/include/tmb/cancellation_token.h
index d860a8a..476791b 100644
--- a/third_party/tmb/include/tmb/cancellation_token.h
+++ b/third_party/tmb/include/tmb/cancellation_token.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_CANCELLATION_TOKEN_H_
 #define TMB_CANCELLATION_TOKEN_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/id_typedefs.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/id_typedefs.h b/third_party/tmb/include/tmb/id_typedefs.h
index 0c57399..d5391c8 100644
--- a/third_party/tmb/include/tmb/id_typedefs.h
+++ b/third_party/tmb/include/tmb/id_typedefs.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_ID_TYPEDEFS_H_
 #define TMB_ID_TYPEDEFS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/c_string_buffer.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/c_string_buffer.h b/third_party/tmb/include/tmb/internal/c_string_buffer.h
index ce0af2f..9aa2f8a 100644
--- a/third_party/tmb/include/tmb/internal/c_string_buffer.h
+++ b/third_party/tmb/include/tmb/internal/c_string_buffer.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_C_STRING_BUFFER_H_
 #define TMB_INTERNAL_C_STRING_BUFFER_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/cache_info.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/cache_info.h b/third_party/tmb/include/tmb/internal/cache_info.h
index 6d4bd7b..b1da840 100644
--- a/third_party/tmb/include/tmb/internal/cache_info.h
+++ b/third_party/tmb/include/tmb/internal/cache_info.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_CACHE_INFO_H_
 #define TMB_INTERNAL_CACHE_INFO_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/container_pusher.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/container_pusher.h b/third_party/tmb/include/tmb/internal/container_pusher.h
index ad453d1..c77c124 100644
--- a/third_party/tmb/include/tmb/internal/container_pusher.h
+++ b/third_party/tmb/include/tmb/internal/container_pusher.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_CONTAINER_PUSHER_H_
 #define TMB_INTERNAL_CONTAINER_PUSHER_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/crc32.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/crc32.h b/third_party/tmb/include/tmb/internal/crc32.h
index ec14b9d..9393aa4 100644
--- a/third_party/tmb/include/tmb/internal/crc32.h
+++ b/third_party/tmb/include/tmb/internal/crc32.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_CRC32_H_
 #define TMB_INTERNAL_CRC32_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/heap_receiver_message_queue.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/heap_receiver_message_queue.h b/third_party/tmb/include/tmb/internal/heap_receiver_message_queue.h
index 9f1d65b..f41534b 100644
--- a/third_party/tmb/include/tmb/internal/heap_receiver_message_queue.h
+++ b/third_party/tmb/include/tmb/internal/heap_receiver_message_queue.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_HEAP_RECEIVER_MESSAGE_QUEUE_H_
 #define TMB_INTERNAL_HEAP_RECEIVER_MESSAGE_QUEUE_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/iterator_adapter.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/iterator_adapter.h b/third_party/tmb/include/tmb/internal/iterator_adapter.h
index 6fd3e74..88318e7 100644
--- a/third_party/tmb/include/tmb/internal/iterator_adapter.h
+++ b/third_party/tmb/include/tmb/internal/iterator_adapter.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_ITERATOR_ADAPTER_H_
 #define TMB_INTERNAL_ITERATOR_ADAPTER_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/leveldb_key_comparator.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/leveldb_key_comparator.h b/third_party/tmb/include/tmb/internal/leveldb_key_comparator.h
index 63d35ed..0e8d809 100644
--- a/third_party/tmb/include/tmb/internal/leveldb_key_comparator.h
+++ b/third_party/tmb/include/tmb/internal/leveldb_key_comparator.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LEVELDB_KEY_COMPARATOR_H_
 #define TMB_INTERNAL_LEVELDB_KEY_COMPARATOR_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/leveldb_keys.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/leveldb_keys.h b/third_party/tmb/include/tmb/internal/leveldb_keys.h
index c639d8e..23d1e26 100644
--- a/third_party/tmb/include/tmb/internal/leveldb_keys.h
+++ b/third_party/tmb/include/tmb/internal/leveldb_keys.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LEVELDB_KEYS_H_
 #define TMB_INTERNAL_LEVELDB_KEYS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/lock_free_garbage_collector.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/lock_free_garbage_collector.h b/third_party/tmb/include/tmb/internal/lock_free_garbage_collector.h
index 1d7c77a..650a6ea 100644
--- a/third_party/tmb/include/tmb/internal/lock_free_garbage_collector.h
+++ b/third_party/tmb/include/tmb/internal/lock_free_garbage_collector.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOCK_FREE_GARBAGE_COLLECTOR_H_
 #define TMB_INTERNAL_LOCK_FREE_GARBAGE_COLLECTOR_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/lock_free_stack.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/lock_free_stack.h b/third_party/tmb/include/tmb/internal/lock_free_stack.h
index 3da52ac..0022f81 100644
--- a/third_party/tmb/include/tmb/internal/lock_free_stack.h
+++ b/third_party/tmb/include/tmb/internal/lock_free_stack.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOCK_FREE_STACK_H_
 #define TMB_INTERNAL_LOCK_FREE_STACK_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_read_status.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_read_status.h b/third_party/tmb/include/tmb/internal/log_read_status.h
index aed5dc4..3f7b2c8 100644
--- a/third_party/tmb/include/tmb/internal/log_read_status.h
+++ b/third_party/tmb/include/tmb/internal/log_read_status.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_READ_STATUS_H_
 #define TMB_INTERNAL_LOG_READ_STATUS_H_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/third_party/tmb/include/tmb/internal/log_reader_base.h
----------------------------------------------------------------------
diff --git a/third_party/tmb/include/tmb/internal/log_reader_base.h b/third_party/tmb/include/tmb/internal/log_reader_base.h
index 4463ea1..0ac1128 100644
--- a/third_party/tmb/include/tmb/internal/log_reader_base.h
+++ b/third_party/tmb/include/tmb/internal/log_reader_base.h
@@ -1,16 +1,21 @@
-//   Copyright 2014-2015 Quickstep Technologies LLC.
-//
-//   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.
+/**
+ * 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 TMB_INTERNAL_LOG_READER_BASE_H_
 #define TMB_INTERNAL_LOG_READER_BASE_H_



[14/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMakeLists.txt b/types/operations/binary_operations/CMakeLists.txt
index 63db377..071e9fb 100644
--- a/types/operations/binary_operations/CMakeLists.txt
+++ b/types/operations/binary_operations/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_types_operations_binaryoperations_AddBinaryOperation

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/DivideBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/DivideBinaryOperation.cpp b/types/operations/binary_operations/DivideBinaryOperation.cpp
index 23cbb99..59dcb3e 100644
--- a/types/operations/binary_operations/DivideBinaryOperation.cpp
+++ b/types/operations/binary_operations/DivideBinaryOperation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/DivideBinaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/DivideBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/DivideBinaryOperation.hpp b/types/operations/binary_operations/DivideBinaryOperation.hpp
index 7195073..3ab7aa1 100644
--- a/types/operations/binary_operations/DivideBinaryOperation.hpp
+++ b/types/operations/binary_operations/DivideBinaryOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_DIVIDE_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/ModuloBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ModuloBinaryOperation.cpp b/types/operations/binary_operations/ModuloBinaryOperation.cpp
index 9af5470..917c6c9 100644
--- a/types/operations/binary_operations/ModuloBinaryOperation.cpp
+++ b/types/operations/binary_operations/ModuloBinaryOperation.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/ModuloBinaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/ModuloBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ModuloBinaryOperation.hpp b/types/operations/binary_operations/ModuloBinaryOperation.hpp
index dcfb2b5..1d47a45 100644
--- a/types/operations/binary_operations/ModuloBinaryOperation.hpp
+++ b/types/operations/binary_operations/ModuloBinaryOperation.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_MODULO_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/MultiplyBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/MultiplyBinaryOperation.cpp b/types/operations/binary_operations/MultiplyBinaryOperation.cpp
index 970953d..a206364 100644
--- a/types/operations/binary_operations/MultiplyBinaryOperation.cpp
+++ b/types/operations/binary_operations/MultiplyBinaryOperation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/MultiplyBinaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/MultiplyBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/MultiplyBinaryOperation.hpp b/types/operations/binary_operations/MultiplyBinaryOperation.hpp
index 6940bd0..6edc999 100644
--- a/types/operations/binary_operations/MultiplyBinaryOperation.hpp
+++ b/types/operations/binary_operations/MultiplyBinaryOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_MULTIPLY_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/SubtractBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/SubtractBinaryOperation.cpp b/types/operations/binary_operations/SubtractBinaryOperation.cpp
index e410488..fe8644a 100644
--- a/types/operations/binary_operations/SubtractBinaryOperation.cpp
+++ b/types/operations/binary_operations/SubtractBinaryOperation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/SubtractBinaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/SubtractBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/SubtractBinaryOperation.hpp b/types/operations/binary_operations/SubtractBinaryOperation.hpp
index 46224d1..8e54362 100644
--- a/types/operations/binary_operations/SubtractBinaryOperation.hpp
+++ b/types/operations/binary_operations/SubtractBinaryOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_SUBTRACT_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/AddBinaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/AddBinaryOperation_unittest.cpp b/types/operations/binary_operations/tests/AddBinaryOperation_unittest.cpp
index fdad57d..e6b0484 100644
--- a/types/operations/binary_operations/tests/AddBinaryOperation_unittest.cpp
+++ b/types/operations/binary_operations/tests/AddBinaryOperation_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <initializer_list>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/BinaryOperationTestUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/BinaryOperationTestUtil.hpp b/types/operations/binary_operations/tests/BinaryOperationTestUtil.hpp
index 86c7557..e23057c 100644
--- a/types/operations/binary_operations/tests/BinaryOperationTestUtil.hpp
+++ b/types/operations/binary_operations/tests/BinaryOperationTestUtil.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_TESTS_BINARY_OPERATION_TEST_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/BinaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/BinaryOperation_unittest.cpp b/types/operations/binary_operations/tests/BinaryOperation_unittest.cpp
index 8e1d140..f7891b0 100644
--- a/types/operations/binary_operations/tests/BinaryOperation_unittest.cpp
+++ b/types/operations/binary_operations/tests/BinaryOperation_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstdint>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/DivideBinaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/DivideBinaryOperation_unittest.cpp b/types/operations/binary_operations/tests/DivideBinaryOperation_unittest.cpp
index 866fc39..7aca09b 100644
--- a/types/operations/binary_operations/tests/DivideBinaryOperation_unittest.cpp
+++ b/types/operations/binary_operations/tests/DivideBinaryOperation_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <initializer_list>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/ModuloBinaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/ModuloBinaryOperation_unittest.cpp b/types/operations/binary_operations/tests/ModuloBinaryOperation_unittest.cpp
index c8e96e0..fa540cb 100644
--- a/types/operations/binary_operations/tests/ModuloBinaryOperation_unittest.cpp
+++ b/types/operations/binary_operations/tests/ModuloBinaryOperation_unittest.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <initializer_list>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/MultiplyBinaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/MultiplyBinaryOperation_unittest.cpp b/types/operations/binary_operations/tests/MultiplyBinaryOperation_unittest.cpp
index 67eaabc..6559e52 100644
--- a/types/operations/binary_operations/tests/MultiplyBinaryOperation_unittest.cpp
+++ b/types/operations/binary_operations/tests/MultiplyBinaryOperation_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <initializer_list>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/tests/SubtractBinaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/tests/SubtractBinaryOperation_unittest.cpp b/types/operations/binary_operations/tests/SubtractBinaryOperation_unittest.cpp
index e865cca..50ac967 100644
--- a/types/operations/binary_operations/tests/SubtractBinaryOperation_unittest.cpp
+++ b/types/operations/binary_operations/tests/SubtractBinaryOperation_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <initializer_list>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/AsciiStringComparators-inl.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/AsciiStringComparators-inl.hpp b/types/operations/comparisons/AsciiStringComparators-inl.hpp
index 69b4ab3..cde03c3 100644
--- a/types/operations/comparisons/AsciiStringComparators-inl.hpp
+++ b/types/operations/comparisons/AsciiStringComparators-inl.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 // NOTE(chasseur): This file helps resolve some otherwise intractible circular

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/AsciiStringComparators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/AsciiStringComparators.hpp b/types/operations/comparisons/AsciiStringComparators.hpp
index d9d9417..936fd1f 100644
--- a/types/operations/comparisons/AsciiStringComparators.hpp
+++ b/types/operations/comparisons/AsciiStringComparators.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_ASCII_STRING_COMPARATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/BasicComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/BasicComparison.cpp b/types/operations/comparisons/BasicComparison.cpp
index ed0ce62..ec46355 100644
--- a/types/operations/comparisons/BasicComparison.cpp
+++ b/types/operations/comparisons/BasicComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/BasicComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/BasicComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/BasicComparison.hpp b/types/operations/comparisons/BasicComparison.hpp
index c95f774..a7dd50a 100644
--- a/types/operations/comparisons/BasicComparison.hpp
+++ b/types/operations/comparisons/BasicComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_BASIC_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/CMakeLists.txt b/types/operations/comparisons/CMakeLists.txt
index d48cdc2..321c0f6 100644
--- a/types/operations/comparisons/CMakeLists.txt
+++ b/types/operations/comparisons/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_types_operations_comparisons_AsciiStringComparators

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/Comparison-inl.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/Comparison-inl.hpp b/types/operations/comparisons/Comparison-inl.hpp
index 00450fc..25f5f15 100644
--- a/types/operations/comparisons/Comparison-inl.hpp
+++ b/types/operations/comparisons/Comparison-inl.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_COMPARATORS_INL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/Comparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/Comparison.cpp b/types/operations/comparisons/Comparison.cpp
index 1b73edf..ef04ccf 100644
--- a/types/operations/comparisons/Comparison.cpp
+++ b/types/operations/comparisons/Comparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/Comparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/Comparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/Comparison.hpp b/types/operations/comparisons/Comparison.hpp
index 0349bff..c300e74 100644
--- a/types/operations/comparisons/Comparison.hpp
+++ b/types/operations/comparisons/Comparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/ComparisonFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonFactory.cpp b/types/operations/comparisons/ComparisonFactory.cpp
index 1c3b21f..5e96dcc 100644
--- a/types/operations/comparisons/ComparisonFactory.cpp
+++ b/types/operations/comparisons/ComparisonFactory.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/ComparisonFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/ComparisonFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonFactory.hpp b/types/operations/comparisons/ComparisonFactory.hpp
index 8157419..33b43d2 100644
--- a/types/operations/comparisons/ComparisonFactory.hpp
+++ b/types/operations/comparisons/ComparisonFactory.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_COMPARISON_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/ComparisonID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonID.cpp b/types/operations/comparisons/ComparisonID.cpp
index 5fa3b83..be8bdcf 100644
--- a/types/operations/comparisons/ComparisonID.cpp
+++ b/types/operations/comparisons/ComparisonID.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/ComparisonID.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/ComparisonID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonID.hpp b/types/operations/comparisons/ComparisonID.hpp
index 7e83935..ff9c72f 100644
--- a/types/operations/comparisons/ComparisonID.hpp
+++ b/types/operations/comparisons/ComparisonID.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_COMPARISON_ID_HPP_
@@ -61,12 +63,12 @@ extern const char *kComparisonShortNames[
         ComparisonID::kNumComparisonIDs)];
 
 /**
- * @brief Flips a comparison. 
- * 
+ * @brief Flips a comparison.
+ *
  * As in greater than flips to less than, less than flips to greater than, and
  * similarly for greater/less than or equals. Notice that flipping equals
  * results in equals, same for not equals.
- * 
+ *
  * @param comparison The Id of a comparison to flip.
  * @return The flipped comparison id.
  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/ComparisonUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonUtil.hpp b/types/operations/comparisons/ComparisonUtil.hpp
index d589192..5d868fc 100644
--- a/types/operations/comparisons/ComparisonUtil.hpp
+++ b/types/operations/comparisons/ComparisonUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_COMPARISON_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/EqualComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/EqualComparison.cpp b/types/operations/comparisons/EqualComparison.cpp
index 1e3008c..176edec 100644
--- a/types/operations/comparisons/EqualComparison.cpp
+++ b/types/operations/comparisons/EqualComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/EqualComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/EqualComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/EqualComparison.hpp b/types/operations/comparisons/EqualComparison.hpp
index b4774f9..a304b20 100644
--- a/types/operations/comparisons/EqualComparison.hpp
+++ b/types/operations/comparisons/EqualComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_EQUAL_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/GreaterComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/GreaterComparison.cpp b/types/operations/comparisons/GreaterComparison.cpp
index 4e716ff..d02841d 100644
--- a/types/operations/comparisons/GreaterComparison.cpp
+++ b/types/operations/comparisons/GreaterComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/GreaterComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/GreaterComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/GreaterComparison.hpp b/types/operations/comparisons/GreaterComparison.hpp
index d098386..89ca4cc 100644
--- a/types/operations/comparisons/GreaterComparison.hpp
+++ b/types/operations/comparisons/GreaterComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_GREATER_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/GreaterOrEqualComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/GreaterOrEqualComparison.cpp b/types/operations/comparisons/GreaterOrEqualComparison.cpp
index 3ef4497..dc08a8b 100644
--- a/types/operations/comparisons/GreaterOrEqualComparison.cpp
+++ b/types/operations/comparisons/GreaterOrEqualComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/GreaterOrEqualComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/GreaterOrEqualComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/GreaterOrEqualComparison.hpp b/types/operations/comparisons/GreaterOrEqualComparison.hpp
index bf15032..0d4060d 100644
--- a/types/operations/comparisons/GreaterOrEqualComparison.hpp
+++ b/types/operations/comparisons/GreaterOrEqualComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_GREATER_OR_EQUAL_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/LessComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LessComparison.cpp b/types/operations/comparisons/LessComparison.cpp
index d3a5e74..dbf7ec5 100644
--- a/types/operations/comparisons/LessComparison.cpp
+++ b/types/operations/comparisons/LessComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/LessComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/LessComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LessComparison.hpp b/types/operations/comparisons/LessComparison.hpp
index a5605f4..b7a1b87 100644
--- a/types/operations/comparisons/LessComparison.hpp
+++ b/types/operations/comparisons/LessComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_LESS_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/LessOrEqualComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LessOrEqualComparison.cpp b/types/operations/comparisons/LessOrEqualComparison.cpp
index ce76cbd..3924b2d 100644
--- a/types/operations/comparisons/LessOrEqualComparison.cpp
+++ b/types/operations/comparisons/LessOrEqualComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/LessOrEqualComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/LessOrEqualComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LessOrEqualComparison.hpp b/types/operations/comparisons/LessOrEqualComparison.hpp
index 19ac652..a30f56c 100644
--- a/types/operations/comparisons/LessOrEqualComparison.hpp
+++ b/types/operations/comparisons/LessOrEqualComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_LESS_OR_EQUAL_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/LiteralComparators-inl.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LiteralComparators-inl.hpp b/types/operations/comparisons/LiteralComparators-inl.hpp
index e40f6a5..14844ac 100644
--- a/types/operations/comparisons/LiteralComparators-inl.hpp
+++ b/types/operations/comparisons/LiteralComparators-inl.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 // NOTE(chasseur): This file helps resolve some otherwise intractible circular

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/LiteralComparators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LiteralComparators.hpp b/types/operations/comparisons/LiteralComparators.hpp
index 406ae40..c7b9b34 100644
--- a/types/operations/comparisons/LiteralComparators.hpp
+++ b/types/operations/comparisons/LiteralComparators.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_LITERAL_COMPARATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/NotEqualComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/NotEqualComparison.cpp b/types/operations/comparisons/NotEqualComparison.cpp
index 235b961..9a728c4 100644
--- a/types/operations/comparisons/NotEqualComparison.cpp
+++ b/types/operations/comparisons/NotEqualComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/NotEqualComparison.hpp"



[23/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index ad3a9ff..7639966 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_TABLE_GENERATOR_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 49c9150..1a0b715 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/TextScanOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 6890d7d..24af844 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_TEXT_SCAN_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index f103b0e..bc29365 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/UpdateOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index a443b5d..c095164 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_UPDATE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/WindowAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WindowAggregationOperator.cpp b/relational_operators/WindowAggregationOperator.cpp
index 3149864..5a1f8ec 100644
--- a/relational_operators/WindowAggregationOperator.cpp
+++ b/relational_operators/WindowAggregationOperator.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/WindowAggregationOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/WindowAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WindowAggregationOperator.hpp b/relational_operators/WindowAggregationOperator.hpp
index 05632cc..530c2c1 100644
--- a/relational_operators/WindowAggregationOperator.hpp
+++ b/relational_operators/WindowAggregationOperator.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_WINDOW_AGGREGATION_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index df195cc..3cbab94 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_WORK_UNIT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 076735f..02aa50e 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//     University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index e078b84..f920cac 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/WorkOrderFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/WorkOrderFactory.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.hpp b/relational_operators/WorkOrderFactory.hpp
index a870d09..45e2cbc 100644
--- a/relational_operators/WorkOrderFactory.hpp
+++ b/relational_operators/WorkOrderFactory.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_WORK_ORDER_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index fd4692a..7a5b461 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 9c34170..239547f 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 46bce5b..b37d81f 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index bd682c2..d09ff07 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 5bcbee5..53a9124 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 4878cf1..3f6e23a 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/AggregationOperationState.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 0199749..ecd116b 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/AggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.proto b/storage/AggregationOperationState.proto
index bf78e3a..7521d73 100644
--- a/storage/AggregationOperationState.proto
+++ b/storage/AggregationOperationState.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//     University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/BasicColumnStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreTupleStorageSubBlock.cpp b/storage/BasicColumnStoreTupleStorageSubBlock.cpp
index a474437..34a1620 100644
--- a/storage/BasicColumnStoreTupleStorageSubBlock.cpp
+++ b/storage/BasicColumnStoreTupleStorageSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/BasicColumnStoreTupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/BasicColumnStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreTupleStorageSubBlock.hpp b/storage/BasicColumnStoreTupleStorageSubBlock.hpp
index b36eb20..e896a7a 100644
--- a/storage/BasicColumnStoreTupleStorageSubBlock.hpp
+++ b/storage/BasicColumnStoreTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/BasicColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreValueAccessor.hpp b/storage/BasicColumnStoreValueAccessor.hpp
index 759e187..0560d99 100644
--- a/storage/BasicColumnStoreValueAccessor.hpp
+++ b/storage/BasicColumnStoreValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/BloomFilterIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.cpp b/storage/BloomFilterIndexSubBlock.cpp
index e806217..4351c05 100644
--- a/storage/BloomFilterIndexSubBlock.cpp
+++ b/storage/BloomFilterIndexSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/BloomFilterIndexSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/BloomFilterIndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.hpp b/storage/BloomFilterIndexSubBlock.hpp
index 4925673..09d6225 100644
--- a/storage/BloomFilterIndexSubBlock.hpp
+++ b/storage/BloomFilterIndexSubBlock.hpp
@@ -1,19 +1,21 @@
 /**
-*   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-*     University of Wisconsin\u2014Madison.
-*
-*   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.
-**/
+ * 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 QUICKSTEP_STORAGE_BLOOM_FILTER_INDEX_SUB_BLOCK_HPP_
 #define QUICKSTEP_STORAGE_BLOOM_FILTER_INDEX_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 582effd..c4b8f70 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 option(REBUILD_INDEX_ON_UPDATE_OVERFLOW "If an IndexSubBlock runs out of space while re-adding entries during an update() query, try rebuilding it." ON)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CSBTreeIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CSBTreeIndexSubBlock.cpp b/storage/CSBTreeIndexSubBlock.cpp
index 8535398..dd75467 100644
--- a/storage/CSBTreeIndexSubBlock.cpp
+++ b/storage/CSBTreeIndexSubBlock.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/CSBTreeIndexSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CSBTreeIndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/CSBTreeIndexSubBlock.hpp b/storage/CSBTreeIndexSubBlock.hpp
index 24e2ab5..4503eff 100644
--- a/storage/CSBTreeIndexSubBlock.hpp
+++ b/storage/CSBTreeIndexSubBlock.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_CSBTREE_INDEX_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/ColumnStoreUtil.cpp
----------------------------------------------------------------------
diff --git a/storage/ColumnStoreUtil.cpp b/storage/ColumnStoreUtil.cpp
index 1355376..b28b576 100644
--- a/storage/ColumnStoreUtil.cpp
+++ b/storage/ColumnStoreUtil.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/ColumnStoreUtil.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/ColumnStoreUtil.hpp
----------------------------------------------------------------------
diff --git a/storage/ColumnStoreUtil.hpp b/storage/ColumnStoreUtil.hpp
index eec452c..7bb6621 100644
--- a/storage/ColumnStoreUtil.hpp
+++ b/storage/ColumnStoreUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COLUMN_STORE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedBlockBuilder.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedBlockBuilder.cpp b/storage/CompressedBlockBuilder.cpp
index 1ca0c07..ef819fc 100644
--- a/storage/CompressedBlockBuilder.cpp
+++ b/storage/CompressedBlockBuilder.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/CompressedBlockBuilder.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedBlockBuilder.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedBlockBuilder.hpp b/storage/CompressedBlockBuilder.hpp
index 0e66de6..a323f6b 100644
--- a/storage/CompressedBlockBuilder.hpp
+++ b/storage/CompressedBlockBuilder.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_BLOCK_BUILDER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreTupleStorageSubBlock.cpp b/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
index 1173a84..3bd0c3a 100644
--- a/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
+++ b/storage/CompressedColumnStoreTupleStorageSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/CompressedColumnStoreTupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedColumnStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreTupleStorageSubBlock.hpp b/storage/CompressedColumnStoreTupleStorageSubBlock.hpp
index ad35430..55c8945 100644
--- a/storage/CompressedColumnStoreTupleStorageSubBlock.hpp
+++ b/storage/CompressedColumnStoreTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_COLUMN_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreValueAccessor.hpp b/storage/CompressedColumnStoreValueAccessor.hpp
index 64eb315..25e5eed 100644
--- a/storage/CompressedColumnStoreValueAccessor.hpp
+++ b/storage/CompressedColumnStoreValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_COLUMN_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp b/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
index 163e7df..d362f98 100644
--- a/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
+++ b/storage/CompressedPackedRowStoreTupleStorageSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp b/storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp
index c9abecd..cb7b9df 100644
--- a/storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp
+++ b/storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedPackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreValueAccessor.hpp b/storage/CompressedPackedRowStoreValueAccessor.hpp
index 024b0ec..8858175 100644
--- a/storage/CompressedPackedRowStoreValueAccessor.hpp
+++ b/storage/CompressedPackedRowStoreValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedStoreUtil.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedStoreUtil.cpp b/storage/CompressedStoreUtil.cpp
index e9ed6a8..46887ec 100644
--- a/storage/CompressedStoreUtil.cpp
+++ b/storage/CompressedStoreUtil.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/CompressedStoreUtil.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedStoreUtil.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedStoreUtil.hpp b/storage/CompressedStoreUtil.hpp
index 019f52c..963cd79 100644
--- a/storage/CompressedStoreUtil.hpp
+++ b/storage/CompressedStoreUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_STORE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/CompressedTupleStorageSubBlock.cpp b/storage/CompressedTupleStorageSubBlock.cpp
index 53379f6..e443d18 100644
--- a/storage/CompressedTupleStorageSubBlock.cpp
+++ b/storage/CompressedTupleStorageSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/CompressedTupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CompressedTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedTupleStorageSubBlock.hpp b/storage/CompressedTupleStorageSubBlock.hpp
index 19ebc7f..926ef4f 100644
--- a/storage/CompressedTupleStorageSubBlock.hpp
+++ b/storage/CompressedTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COMPRESSED_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/CountedReference.hpp
----------------------------------------------------------------------
diff --git a/storage/CountedReference.hpp b/storage/CountedReference.hpp
index 10a8a87..2d9cec3 100644
--- a/storage/CountedReference.hpp
+++ b/storage/CountedReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_COUNTED_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/DataExchange.proto
----------------------------------------------------------------------
diff --git a/storage/DataExchange.proto b/storage/DataExchange.proto
index a2636e5..1cc6024 100644
--- a/storage/DataExchange.proto
+++ b/storage/DataExchange.proto
@@ -1,16 +1,19 @@
-//   Copyright 2016 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto3";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/DataExchangerAsync.cpp
----------------------------------------------------------------------
diff --git a/storage/DataExchangerAsync.cpp b/storage/DataExchangerAsync.cpp
index 78c6565..59f5ebf 100644
--- a/storage/DataExchangerAsync.cpp
+++ b/storage/DataExchangerAsync.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/DataExchangerAsync.hpp"



[29/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/LogicalOr.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/LogicalOr.hpp b/query_optimizer/expressions/LogicalOr.hpp
index 0dca433..f96b6cd 100644
--- a/query_optimizer/expressions/LogicalOr.hpp
+++ b/query_optimizer/expressions/LogicalOr.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_LOGICAL_OR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/NamedExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/NamedExpression.cpp b/query_optimizer/expressions/NamedExpression.cpp
index 352f33d..992a84a 100644
--- a/query_optimizer/expressions/NamedExpression.cpp
+++ b/query_optimizer/expressions/NamedExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/NamedExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/NamedExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/NamedExpression.hpp b/query_optimizer/expressions/NamedExpression.hpp
index be3176b..9de8005 100644
--- a/query_optimizer/expressions/NamedExpression.hpp
+++ b/query_optimizer/expressions/NamedExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_NAMED_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/OptimizerExpressionsModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/OptimizerExpressionsModule.hpp b/query_optimizer/expressions/OptimizerExpressionsModule.hpp
index 2761ba8..4462201 100644
--- a/query_optimizer/expressions/OptimizerExpressionsModule.hpp
+++ b/query_optimizer/expressions/OptimizerExpressionsModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup OptimizerExpressions

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/PatternMatcher.hpp b/query_optimizer/expressions/PatternMatcher.hpp
index 2cc91d6..18d6b1c 100644
--- a/query_optimizer/expressions/PatternMatcher.hpp
+++ b/query_optimizer/expressions/PatternMatcher.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_PATTERN_MATCHER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Predicate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Predicate.hpp b/query_optimizer/expressions/Predicate.hpp
index ac706e6..444af6b 100644
--- a/query_optimizer/expressions/Predicate.hpp
+++ b/query_optimizer/expressions/Predicate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/PredicateLiteral.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/PredicateLiteral.cpp b/query_optimizer/expressions/PredicateLiteral.cpp
index 54299b3..724458e 100644
--- a/query_optimizer/expressions/PredicateLiteral.cpp
+++ b/query_optimizer/expressions/PredicateLiteral.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/PredicateLiteral.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/PredicateLiteral.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/PredicateLiteral.hpp b/query_optimizer/expressions/PredicateLiteral.hpp
index 256b3a0..234b6e1 100644
--- a/query_optimizer/expressions/PredicateLiteral.hpp
+++ b/query_optimizer/expressions/PredicateLiteral.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_PREDICATE_LITERAL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/Scalar.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Scalar.hpp b/query_optimizer/expressions/Scalar.hpp
index 57db4e2..4870118 100644
--- a/query_optimizer/expressions/Scalar.hpp
+++ b/query_optimizer/expressions/Scalar.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SCALAR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ScalarLiteral.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ScalarLiteral.cpp b/query_optimizer/expressions/ScalarLiteral.cpp
index 7a1be30..d70c4cf 100644
--- a/query_optimizer/expressions/ScalarLiteral.cpp
+++ b/query_optimizer/expressions/ScalarLiteral.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/ScalarLiteral.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/ScalarLiteral.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ScalarLiteral.hpp b/query_optimizer/expressions/ScalarLiteral.hpp
index 7ebb0d9..8ebc41c 100644
--- a/query_optimizer/expressions/ScalarLiteral.hpp
+++ b/query_optimizer/expressions/ScalarLiteral.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SCALAR_LITERAL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/SearchedCase.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SearchedCase.cpp b/query_optimizer/expressions/SearchedCase.cpp
index 57da7a1..2de52c9 100644
--- a/query_optimizer/expressions/SearchedCase.cpp
+++ b/query_optimizer/expressions/SearchedCase.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/SearchedCase.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/SearchedCase.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SearchedCase.hpp b/query_optimizer/expressions/SearchedCase.hpp
index 8bea0c4..79c37a5 100644
--- a/query_optimizer/expressions/SearchedCase.hpp
+++ b/query_optimizer/expressions/SearchedCase.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SEARCHED_CASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/SimpleCase.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SimpleCase.cpp b/query_optimizer/expressions/SimpleCase.cpp
index 8c6227f..454d7b9 100644
--- a/query_optimizer/expressions/SimpleCase.cpp
+++ b/query_optimizer/expressions/SimpleCase.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/SimpleCase.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/SimpleCase.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SimpleCase.hpp b/query_optimizer/expressions/SimpleCase.hpp
index a037a0e..897d87f 100644
--- a/query_optimizer/expressions/SimpleCase.hpp
+++ b/query_optimizer/expressions/SimpleCase.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SIMPLE_CASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/SubqueryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SubqueryExpression.cpp b/query_optimizer/expressions/SubqueryExpression.cpp
index 3a33fe1..dbcb6d6 100644
--- a/query_optimizer/expressions/SubqueryExpression.cpp
+++ b/query_optimizer/expressions/SubqueryExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/SubqueryExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/SubqueryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SubqueryExpression.hpp b/query_optimizer/expressions/SubqueryExpression.hpp
index 58fb6e8..a128e7f 100644
--- a/query_optimizer/expressions/SubqueryExpression.hpp
+++ b/query_optimizer/expressions/SubqueryExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SUBQUERY_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/UnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.cpp b/query_optimizer/expressions/UnaryExpression.cpp
index ae88d57..b0fff62 100644
--- a/query_optimizer/expressions/UnaryExpression.cpp
+++ b/query_optimizer/expressions/UnaryExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/UnaryExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/UnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.hpp b/query_optimizer/expressions/UnaryExpression.hpp
index 5d4bef0..c4542d0 100644
--- a/query_optimizer/expressions/UnaryExpression.hpp
+++ b/query_optimizer/expressions/UnaryExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_UNARY_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/WindowAggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/WindowAggregateFunction.cpp b/query_optimizer/expressions/WindowAggregateFunction.cpp
index be5db59..bba7c0e 100644
--- a/query_optimizer/expressions/WindowAggregateFunction.cpp
+++ b/query_optimizer/expressions/WindowAggregateFunction.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/expressions/WindowAggregateFunction.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/expressions/WindowAggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/WindowAggregateFunction.hpp b/query_optimizer/expressions/WindowAggregateFunction.hpp
index 0cc80df..04b4b51 100644
--- a/query_optimizer/expressions/WindowAggregateFunction.hpp
+++ b/query_optimizer/expressions/WindowAggregateFunction.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_WINDOW_AGGREGATE_FUNCTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Aggregate.cpp b/query_optimizer/logical/Aggregate.cpp
index 27bbd1b..6559aca 100644
--- a/query_optimizer/logical/Aggregate.cpp
+++ b/query_optimizer/logical/Aggregate.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/Aggregate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Aggregate.hpp b/query_optimizer/logical/Aggregate.hpp
index 78bd9f3..d8c02ba 100644
--- a/query_optimizer/logical/Aggregate.hpp
+++ b/query_optimizer/logical/Aggregate.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/BinaryJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/BinaryJoin.cpp b/query_optimizer/logical/BinaryJoin.cpp
index fb8b956..5d859b6 100644
--- a/query_optimizer/logical/BinaryJoin.cpp
+++ b/query_optimizer/logical/BinaryJoin.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/BinaryJoin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/BinaryJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/BinaryJoin.hpp b/query_optimizer/logical/BinaryJoin.hpp
index 4778ba0..ca68ced 100644
--- a/query_optimizer/logical/BinaryJoin.hpp
+++ b/query_optimizer/logical/BinaryJoin.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_BINARY_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CMakeLists.txt b/query_optimizer/logical/CMakeLists.txt
index b787c60..c67f96f 100644
--- a/query_optimizer/logical/CMakeLists.txt
+++ b/query_optimizer/logical/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_queryoptimizer_logical_Aggregate Aggregate.cpp Aggregate.hpp)
@@ -202,7 +202,7 @@ target_link_libraries(quickstep_queryoptimizer_logical_Sample
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_logical_LogicalType
                       quickstep_utility_Cast
-                      quickstep_utility_Macros)                      
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_logical_SharedSubplanReference
                       glog
                       quickstep_queryoptimizer_OptimizerTree

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CopyFrom.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CopyFrom.cpp b/query_optimizer/logical/CopyFrom.cpp
index fb33848..b0a1423 100644
--- a/query_optimizer/logical/CopyFrom.cpp
+++ b/query_optimizer/logical/CopyFrom.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/CopyFrom.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CopyFrom.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CopyFrom.hpp b/query_optimizer/logical/CopyFrom.hpp
index dbfa517..7c5907f 100644
--- a/query_optimizer/logical/CopyFrom.hpp
+++ b/query_optimizer/logical/CopyFrom.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_COPYFROM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CreateIndex.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateIndex.cpp b/query_optimizer/logical/CreateIndex.cpp
index 27cb64f..0166716 100644
--- a/query_optimizer/logical/CreateIndex.cpp
+++ b/query_optimizer/logical/CreateIndex.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/CreateIndex.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CreateIndex.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateIndex.hpp b/query_optimizer/logical/CreateIndex.hpp
index 9a9b87e..4ad762d 100644
--- a/query_optimizer/logical/CreateIndex.hpp
+++ b/query_optimizer/logical/CreateIndex.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_CREATEINDEX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CreateTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateTable.cpp b/query_optimizer/logical/CreateTable.cpp
index 6670539..111d04b 100644
--- a/query_optimizer/logical/CreateTable.cpp
+++ b/query_optimizer/logical/CreateTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/CreateTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/CreateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateTable.hpp b/query_optimizer/logical/CreateTable.hpp
index 327032b..cc7c6d7 100644
--- a/query_optimizer/logical/CreateTable.hpp
+++ b/query_optimizer/logical/CreateTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_CREATETABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/DeleteTuples.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/DeleteTuples.cpp b/query_optimizer/logical/DeleteTuples.cpp
index 9993c49..f62e270 100644
--- a/query_optimizer/logical/DeleteTuples.cpp
+++ b/query_optimizer/logical/DeleteTuples.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/DeleteTuples.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/DeleteTuples.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/DeleteTuples.hpp b/query_optimizer/logical/DeleteTuples.hpp
index d7e7c4e..8ecd82f 100644
--- a/query_optimizer/logical/DeleteTuples.hpp
+++ b/query_optimizer/logical/DeleteTuples.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_DELETE_TUPLES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/DropTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/DropTable.cpp b/query_optimizer/logical/DropTable.cpp
index aadb4d0..4936dfa 100644
--- a/query_optimizer/logical/DropTable.cpp
+++ b/query_optimizer/logical/DropTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/DropTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/DropTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/DropTable.hpp b/query_optimizer/logical/DropTable.hpp
index 9e0dfff..ce70256 100644
--- a/query_optimizer/logical/DropTable.hpp
+++ b/query_optimizer/logical/DropTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_DROPTABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Filter.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Filter.cpp b/query_optimizer/logical/Filter.cpp
index 05f02ed..4346b89 100644
--- a/query_optimizer/logical/Filter.cpp
+++ b/query_optimizer/logical/Filter.cpp
@@ -1,21 +1,22 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/Filter.hpp"
 
 #include <string>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Filter.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Filter.hpp b/query_optimizer/logical/Filter.hpp
index fd0016c..62623e2 100644
--- a/query_optimizer/logical/Filter.hpp
+++ b/query_optimizer/logical/Filter.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_FILTER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/HashJoin.hpp b/query_optimizer/logical/HashJoin.hpp
index 9052a23..b0e63e7 100644
--- a/query_optimizer/logical/HashJoin.hpp
+++ b/query_optimizer/logical/HashJoin.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_HASH_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/InsertSelection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertSelection.cpp b/query_optimizer/logical/InsertSelection.cpp
index 2744b44..3bcbc20 100644
--- a/query_optimizer/logical/InsertSelection.cpp
+++ b/query_optimizer/logical/InsertSelection.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/InsertSelection.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/InsertSelection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertSelection.hpp b/query_optimizer/logical/InsertSelection.hpp
index c36ecd0..9cdbb1c 100644
--- a/query_optimizer/logical/InsertSelection.hpp
+++ b/query_optimizer/logical/InsertSelection.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_INSERT_SELECTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/InsertTuple.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertTuple.cpp b/query_optimizer/logical/InsertTuple.cpp
index b053b33..e5ffa35 100644
--- a/query_optimizer/logical/InsertTuple.cpp
+++ b/query_optimizer/logical/InsertTuple.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/logical/InsertTuple.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/InsertTuple.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/InsertTuple.hpp b/query_optimizer/logical/InsertTuple.hpp
index 2ef647d..fd0301e 100644
--- a/query_optimizer/logical/InsertTuple.hpp
+++ b/query_optimizer/logical/InsertTuple.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_INSERT_TUPLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/logical/Join.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/Join.hpp b/query_optimizer/logical/Join.hpp
index 689dfa2..ae74816 100644
--- a/query_optimizer/logical/Join.hpp
+++ b/query_optimizer/logical/Join.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_JOIN_HPP_



[35/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionFactory.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionFactory.cpp b/expressions/aggregation/AggregateFunctionFactory.cpp
index f973cc3..faedac3 100644
--- a/expressions/aggregation/AggregateFunctionFactory.cpp
+++ b/expressions/aggregation/AggregateFunctionFactory.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunctionFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionFactory.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionFactory.hpp b/expressions/aggregation/AggregateFunctionFactory.hpp
index 0553542..89d3540 100644
--- a/expressions/aggregation/AggregateFunctionFactory.hpp
+++ b/expressions/aggregation/AggregateFunctionFactory.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionMax.cpp b/expressions/aggregation/AggregateFunctionMax.cpp
index 7d3290a..cc04bf4 100644
--- a/expressions/aggregation/AggregateFunctionMax.cpp
+++ b/expressions/aggregation/AggregateFunctionMax.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunctionMax.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionMax.hpp b/expressions/aggregation/AggregateFunctionMax.hpp
index ffbddb3..4eadaa2 100644
--- a/expressions/aggregation/AggregateFunctionMax.hpp
+++ b/expressions/aggregation/AggregateFunctionMax.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_MAX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionMin.cpp b/expressions/aggregation/AggregateFunctionMin.cpp
index 4a24d58..9fa93de 100644
--- a/expressions/aggregation/AggregateFunctionMin.cpp
+++ b/expressions/aggregation/AggregateFunctionMin.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunctionMin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionMin.hpp b/expressions/aggregation/AggregateFunctionMin.hpp
index 3d9b558..90d1762 100644
--- a/expressions/aggregation/AggregateFunctionMin.hpp
+++ b/expressions/aggregation/AggregateFunctionMin.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_MIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionSum.cpp b/expressions/aggregation/AggregateFunctionSum.cpp
index e56104d..b62660f 100644
--- a/expressions/aggregation/AggregateFunctionSum.cpp
+++ b/expressions/aggregation/AggregateFunctionSum.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregateFunctionSum.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregateFunctionSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionSum.hpp b/expressions/aggregation/AggregateFunctionSum.hpp
index 5952835..a4086c3 100644
--- a/expressions/aggregation/AggregateFunctionSum.hpp
+++ b/expressions/aggregation/AggregateFunctionSum.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATE_FUNCTION_SUM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index 05ca58d..719920f 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationConcreteHandle.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index 0267e17..c5ca061 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index cdebb03..3d6e872 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 42a2fb9..4bd43d6 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationHandleAvg.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 4ad4b21..31997b1 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_AVG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index 964b7c2..dfcf131 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationHandleCount.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 50138b9..1cd5bda 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_COUNT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
index fe8ffcf..68fcd4c 100644
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ b/expressions/aggregation/AggregationHandleDistinct.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationHandleDistinct.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index 6342c2b..8524fcc 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index a7a4a52..435f5f2 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationHandleMax.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index 5af5a12..7e38473 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_MAX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index ca9b163..e860d8d 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationHandleMin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index f68bb9d..924698c 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_MIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 691ff39..b5036a8 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/aggregation/AggregationHandleSum.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index fdc0884..3382646 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_SUM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/AggregationID.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationID.hpp b/expressions/aggregation/AggregationID.hpp
index 4f28f4a..1efb35c 100644
--- a/expressions/aggregation/AggregationID.hpp
+++ b/expressions/aggregation/AggregationID.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 5744c52..888d95c 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 QS_PROTOBUF_GENERATE_CPP(expressions_aggregation_AggregateFunction_proto_srcs
                          expressions_aggregation_AggregateFunction_proto_hdrs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
index fd82cba..afc02ec 100644
--- a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
index bf02523..6565a41 100644
--- a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
index fc25e91..b7cf02a 100644
--- a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
index a87ace9..85c3bf3 100644
--- a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
index abf8a89..0e35151 100644
--- a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/predicate/CMakeLists.txt b/expressions/predicate/CMakeLists.txt
index 4838421..b90562c 100644
--- a/expressions/predicate/CMakeLists.txt
+++ b/expressions/predicate/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_expressions_predicate_ComparisonPredicate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/ComparisonPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/ComparisonPredicate.cpp b/expressions/predicate/ComparisonPredicate.cpp
index 8837826..5f8612e 100644
--- a/expressions/predicate/ComparisonPredicate.cpp
+++ b/expressions/predicate/ComparisonPredicate.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/predicate/ComparisonPredicate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/ComparisonPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/ComparisonPredicate.hpp b/expressions/predicate/ComparisonPredicate.hpp
index 7193c5d..9030857 100644
--- a/expressions/predicate/ComparisonPredicate.hpp
+++ b/expressions/predicate/ComparisonPredicate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_COMPARISON_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/ConjunctionPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/ConjunctionPredicate.cpp b/expressions/predicate/ConjunctionPredicate.cpp
index d8756c6..4f5cc77 100644
--- a/expressions/predicate/ConjunctionPredicate.cpp
+++ b/expressions/predicate/ConjunctionPredicate.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/predicate/ConjunctionPredicate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/ConjunctionPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/ConjunctionPredicate.hpp b/expressions/predicate/ConjunctionPredicate.hpp
index a3276f6..b24036a 100644
--- a/expressions/predicate/ConjunctionPredicate.hpp
+++ b/expressions/predicate/ConjunctionPredicate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_CONJUNCTION_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/DisjunctionPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/DisjunctionPredicate.cpp b/expressions/predicate/DisjunctionPredicate.cpp
index 6082748..e117c99 100644
--- a/expressions/predicate/DisjunctionPredicate.cpp
+++ b/expressions/predicate/DisjunctionPredicate.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/predicate/DisjunctionPredicate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/DisjunctionPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/DisjunctionPredicate.hpp b/expressions/predicate/DisjunctionPredicate.hpp
index cc6b209..2127573 100644
--- a/expressions/predicate/DisjunctionPredicate.hpp
+++ b/expressions/predicate/DisjunctionPredicate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_DISJUNCTION_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/NegationPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/NegationPredicate.cpp b/expressions/predicate/NegationPredicate.cpp
index 6b0b806..bee1c8d 100644
--- a/expressions/predicate/NegationPredicate.cpp
+++ b/expressions/predicate/NegationPredicate.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/predicate/NegationPredicate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/NegationPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/NegationPredicate.hpp b/expressions/predicate/NegationPredicate.hpp
index e24317c..33c6df8 100644
--- a/expressions/predicate/NegationPredicate.hpp
+++ b/expressions/predicate/NegationPredicate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_NEGATION_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/Predicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/Predicate.cpp b/expressions/predicate/Predicate.cpp
index f9b1f5e..006e8f1 100644
--- a/expressions/predicate/Predicate.cpp
+++ b/expressions/predicate/Predicate.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/predicate/Predicate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/Predicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/Predicate.hpp b/expressions/predicate/Predicate.hpp
index edcfbdc..5fb3ef5 100644
--- a/expressions/predicate/Predicate.hpp
+++ b/expressions/predicate/Predicate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/PredicateCost.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/PredicateCost.hpp b/expressions/predicate/PredicateCost.hpp
index b83f64c..60e6721 100644
--- a/expressions/predicate/PredicateCost.hpp
+++ b/expressions/predicate/PredicateCost.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_COST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/PredicateWithList.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/PredicateWithList.hpp b/expressions/predicate/PredicateWithList.hpp
index f742ec3..b1bf7e5 100644
--- a/expressions/predicate/PredicateWithList.hpp
+++ b/expressions/predicate/PredicateWithList.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_WITH_LIST_HPP_



[26/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/PushDownSemiAntiJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownSemiAntiJoin.cpp b/query_optimizer/rules/PushDownSemiAntiJoin.cpp
index d53970e..7c54edf 100644
--- a/query_optimizer/rules/PushDownSemiAntiJoin.cpp
+++ b/query_optimizer/rules/PushDownSemiAntiJoin.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/PushDownSemiAntiJoin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/PushDownSemiAntiJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownSemiAntiJoin.hpp b/query_optimizer/rules/PushDownSemiAntiJoin.hpp
index 7b1ea85..8374d80 100644
--- a/query_optimizer/rules/PushDownSemiAntiJoin.hpp
+++ b/query_optimizer/rules/PushDownSemiAntiJoin.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_SEMI_ANTI_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/Rule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/Rule.hpp b/query_optimizer/rules/Rule.hpp
index 7e0e98e..9f4f30a 100644
--- a/query_optimizer/rules/Rule.hpp
+++ b/query_optimizer/rules/Rule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_RULE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/RuleHelper.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/RuleHelper.cpp b/query_optimizer/rules/RuleHelper.cpp
index a273004..84de28d 100644
--- a/query_optimizer/rules/RuleHelper.cpp
+++ b/query_optimizer/rules/RuleHelper.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <vector>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/RuleHelper.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/RuleHelper.hpp b/query_optimizer/rules/RuleHelper.hpp
index 4b7bb16..73f6681 100644
--- a/query_optimizer/rules/RuleHelper.hpp
+++ b/query_optimizer/rules/RuleHelper.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_RULE_HELPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 9770606..946d316 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index deddffd..4d6765c 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_STAR_SCHEMA_HASH_JOIN_ORDER_OPTIMIZATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/SwapProbeBuild.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/SwapProbeBuild.cpp b/query_optimizer/rules/SwapProbeBuild.cpp
index 4b4b38f..cc3f1e2 100644
--- a/query_optimizer/rules/SwapProbeBuild.cpp
+++ b/query_optimizer/rules/SwapProbeBuild.cpp
@@ -1,3 +1,22 @@
+/**
+ * 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 "query_optimizer/rules/SwapProbeBuild.hpp"
 
 #include <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/SwapProbeBuild.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/SwapProbeBuild.hpp b/query_optimizer/rules/SwapProbeBuild.hpp
index da8fdf9..ad756fb 100644
--- a/query_optimizer/rules/SwapProbeBuild.hpp
+++ b/query_optimizer/rules/SwapProbeBuild.hpp
@@ -1,3 +1,22 @@
+/**
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_SWAP_PROBE_BUILD_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_RULES_SWAP_PROBE_BUILD_HPP_
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/TopDownRule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/TopDownRule.hpp b/query_optimizer/rules/TopDownRule.hpp
index 81c3ac3..c54cb11 100644
--- a/query_optimizer/rules/TopDownRule.hpp
+++ b/query_optimizer/rules/TopDownRule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_TOP_DOWN_RULE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/UnnestSubqueries.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/UnnestSubqueries.cpp b/query_optimizer/rules/UnnestSubqueries.cpp
index d22ab8e..b376b0b 100644
--- a/query_optimizer/rules/UnnestSubqueries.cpp
+++ b/query_optimizer/rules/UnnestSubqueries.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/UnnestSubqueries.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/UnnestSubqueries.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/UnnestSubqueries.hpp b/query_optimizer/rules/UnnestSubqueries.hpp
index d51dfd5..45c8771 100644
--- a/query_optimizer/rules/UnnestSubqueries.hpp
+++ b/query_optimizer/rules/UnnestSubqueries.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_UNNEST_SUBQUERIES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/UpdateExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/UpdateExpression.cpp b/query_optimizer/rules/UpdateExpression.cpp
index b99178c..6bff3c1 100644
--- a/query_optimizer/rules/UpdateExpression.cpp
+++ b/query_optimizer/rules/UpdateExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/UpdateExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/UpdateExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/UpdateExpression.hpp b/query_optimizer/rules/UpdateExpression.hpp
index 0c9fe5a..4737b37 100644
--- a/query_optimizer/rules/UpdateExpression.hpp
+++ b/query_optimizer/rules/UpdateExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_UPDATE_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/CMakeLists.txt b/query_optimizer/rules/tests/CMakeLists.txt
index eb0a639..0d913e2 100644
--- a/query_optimizer/rules/tests/CMakeLists.txt
+++ b/query_optimizer/rules/tests/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_library(quickstep_queryoptimizer_rules_tests_ExpressionRuleTest ../../../empty_src.cpp ExpressionRuleTest.hpp)
 add_library(quickstep_queryoptimizer_rules_tests_LogicalRuleTest ../../../empty_src.cpp LogicalRuleTest.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/CollapseProject_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/CollapseProject_unittest.cpp b/query_optimizer/rules/tests/CollapseProject_unittest.cpp
index 17ee7b0..5dd224d 100644
--- a/query_optimizer/rules/tests/CollapseProject_unittest.cpp
+++ b/query_optimizer/rules/tests/CollapseProject_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/CollapseProject.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/ExpressionRuleTest.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/ExpressionRuleTest.hpp b/query_optimizer/rules/tests/ExpressionRuleTest.hpp
index e32ff0c..cb0a691 100644
--- a/query_optimizer/rules/tests/ExpressionRuleTest.hpp
+++ b/query_optimizer/rules/tests/ExpressionRuleTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_TESTS_EXPRESSION_RULE_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/GenerateJoins_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/GenerateJoins_unittest.cpp b/query_optimizer/rules/tests/GenerateJoins_unittest.cpp
index b7db431..ca3d611 100644
--- a/query_optimizer/rules/tests/GenerateJoins_unittest.cpp
+++ b/query_optimizer/rules/tests/GenerateJoins_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/GenerateJoins.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/LogicalRuleTest.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/LogicalRuleTest.hpp b/query_optimizer/rules/tests/LogicalRuleTest.hpp
index 211e83b..be76917 100644
--- a/query_optimizer/rules/tests/LogicalRuleTest.hpp
+++ b/query_optimizer/rules/tests/LogicalRuleTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_TESTS_LOGICAL_RULE_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/PhysicalRuleTest.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/PhysicalRuleTest.hpp b/query_optimizer/rules/tests/PhysicalRuleTest.hpp
index 0e9778b..e02b7d4 100644
--- a/query_optimizer/rules/tests/PhysicalRuleTest.hpp
+++ b/query_optimizer/rules/tests/PhysicalRuleTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_TESTS_PHYSICAL_RULE_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/PruneColumns_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/PruneColumns_unittest.cpp b/query_optimizer/rules/tests/PruneColumns_unittest.cpp
index ff51abf..8781750 100644
--- a/query_optimizer/rules/tests/PruneColumns_unittest.cpp
+++ b/query_optimizer/rules/tests/PruneColumns_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/PruneColumns.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/PushDownFilter_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/PushDownFilter_unittest.cpp b/query_optimizer/rules/tests/PushDownFilter_unittest.cpp
index d2d2db3..9f26180 100644
--- a/query_optimizer/rules/tests/PushDownFilter_unittest.cpp
+++ b/query_optimizer/rules/tests/PushDownFilter_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/PushDownFilter.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/RuleTest.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/RuleTest.hpp b/query_optimizer/rules/tests/RuleTest.hpp
index f5efbe8..3f31821 100644
--- a/query_optimizer/rules/tests/RuleTest.hpp
+++ b/query_optimizer/rules/tests/RuleTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_TESTS_RULE_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/tests/UpdateExpression_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/UpdateExpression_unittest.cpp b/query_optimizer/rules/tests/UpdateExpression_unittest.cpp
index 1b3654b..f13b685 100644
--- a/query_optimizer/rules/tests/UpdateExpression_unittest.cpp
+++ b/query_optimizer/rules/tests/UpdateExpression_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/UpdateExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Aggregate.cpp b/query_optimizer/strategy/Aggregate.cpp
index a05d37e..6476b2c 100644
--- a/query_optimizer/strategy/Aggregate.cpp
+++ b/query_optimizer/strategy/Aggregate.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/Aggregate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Aggregate.hpp b/query_optimizer/strategy/Aggregate.hpp
index 5747413..3aab554 100644
--- a/query_optimizer/strategy/Aggregate.hpp
+++ b/query_optimizer/strategy/Aggregate.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_STRATEGY_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/CMakeLists.txt b/query_optimizer/strategy/CMakeLists.txt
index 517bea3..002fa9b 100644
--- a/query_optimizer/strategy/CMakeLists.txt
+++ b/query_optimizer/strategy/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_subdirectory(tests)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Join.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Join.cpp b/query_optimizer/strategy/Join.cpp
index ceb3f4f..cd01bd1 100644
--- a/query_optimizer/strategy/Join.cpp
+++ b/query_optimizer/strategy/Join.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/Join.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Join.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Join.hpp b/query_optimizer/strategy/Join.hpp
index efba286..ad5d5b2 100644
--- a/query_optimizer/strategy/Join.hpp
+++ b/query_optimizer/strategy/Join.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_STRATEGY_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index e9a8897..78003f4 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/OneToOne.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/OneToOne.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.hpp b/query_optimizer/strategy/OneToOne.hpp
index 04cff65..f295511 100644
--- a/query_optimizer/strategy/OneToOne.hpp
+++ b/query_optimizer/strategy/OneToOne.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_STRATEGY_ONE_TO_ONE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/OptimizerStrategyModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OptimizerStrategyModule.hpp b/query_optimizer/strategy/OptimizerStrategyModule.hpp
index 46eb5b8..21590fa 100644
--- a/query_optimizer/strategy/OptimizerStrategyModule.hpp
+++ b/query_optimizer/strategy/OptimizerStrategyModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup OptimizerStrategy

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Selection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Selection.cpp b/query_optimizer/strategy/Selection.cpp
index 2416448..2fbcdec 100644
--- a/query_optimizer/strategy/Selection.cpp
+++ b/query_optimizer/strategy/Selection.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/Selection.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Selection.hpp b/query_optimizer/strategy/Selection.hpp
index 8e4ee73..a839ae1 100644
--- a/query_optimizer/strategy/Selection.hpp
+++ b/query_optimizer/strategy/Selection.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_STRATEGY_SELECTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/Strategy.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/Strategy.hpp b/query_optimizer/strategy/Strategy.hpp
index 3dc027a..330fc45 100644
--- a/query_optimizer/strategy/Strategy.hpp
+++ b/query_optimizer/strategy/Strategy.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_STRATEGY_STRATEGY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/tests/Aggregate_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/Aggregate_unittest.cpp b/query_optimizer/strategy/tests/Aggregate_unittest.cpp
index 1f4eb95..8eb5e93 100644
--- a/query_optimizer/strategy/tests/Aggregate_unittest.cpp
+++ b/query_optimizer/strategy/tests/Aggregate_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/Aggregate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/CMakeLists.txt b/query_optimizer/strategy/tests/CMakeLists.txt
index e9c1150..97675f0 100644
--- a/query_optimizer/strategy/tests/CMakeLists.txt
+++ b/query_optimizer/strategy/tests/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_library(quickstep_queryoptimizer_strategy_tests_StrategyTest ../../../empty_src.cpp StrategyTest.hpp)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/tests/Join_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/Join_unittest.cpp b/query_optimizer/strategy/tests/Join_unittest.cpp
index bbfee05..87a8a97 100644
--- a/query_optimizer/strategy/tests/Join_unittest.cpp
+++ b/query_optimizer/strategy/tests/Join_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/Join.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/tests/OneToOne_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/OneToOne_unittest.cpp b/query_optimizer/strategy/tests/OneToOne_unittest.cpp
index 957d2d4..9dda602 100644
--- a/query_optimizer/strategy/tests/OneToOne_unittest.cpp
+++ b/query_optimizer/strategy/tests/OneToOne_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/OneToOne.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/tests/Selection_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/Selection_unittest.cpp b/query_optimizer/strategy/tests/Selection_unittest.cpp
index f6679de..35b0bd2 100644
--- a/query_optimizer/strategy/tests/Selection_unittest.cpp
+++ b/query_optimizer/strategy/tests/Selection_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/strategy/Selection.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/strategy/tests/StrategyTest.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/StrategyTest.hpp b/query_optimizer/strategy/tests/StrategyTest.hpp
index 2204211..dcad010 100644
--- a/query_optimizer/strategy/tests/StrategyTest.hpp
+++ b/query_optimizer/strategy/tests/StrategyTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_STRATEGY_TESTS_STRATEGY_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt b/query_optimizer/tests/CMakeLists.txt
index 9cad47f..1453291 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_subdirectory(execution_generator)
 add_subdirectory(logical_generator)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/ExecutionGeneratorTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTest.cpp b/query_optimizer/tests/ExecutionGeneratorTest.cpp
index 42b246b..1101896 100644
--- a/query_optimizer/tests/ExecutionGeneratorTest.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <fstream>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index 195ca59..2e01c7b 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/tests/ExecutionGeneratorTestRunner.hpp"



[25/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
index d1d9380..b8cd02a 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_TESTS_EXECUTION_GENERATOR_TEST_RUNNER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index 815c13e..73b3e84 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/OptimizerTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTest.cpp b/query_optimizer/tests/OptimizerTest.cpp
index 57e2d67..a93db3e 100644
--- a/query_optimizer/tests/OptimizerTest.cpp
+++ b/query_optimizer/tests/OptimizerTest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/tests/OptimizerTest.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/OptimizerTest.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTest.hpp b/query_optimizer/tests/OptimizerTest.hpp
index 78d5a51..4a0491b 100644
--- a/query_optimizer/tests/OptimizerTest.hpp
+++ b/query_optimizer/tests/OptimizerTest.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_TESTS_OPTIMIZER_TEST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/OptimizerTextTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTest.cpp b/query_optimizer/tests/OptimizerTextTest.cpp
index f3c243b..b6be739 100644
--- a/query_optimizer/tests/OptimizerTextTest.cpp
+++ b/query_optimizer/tests/OptimizerTextTest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <fstream>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/OptimizerTextTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.cpp b/query_optimizer/tests/OptimizerTextTestRunner.cpp
index d790c33..93f3123 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.cpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/tests/OptimizerTextTestRunner.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/OptimizerTextTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.hpp b/query_optimizer/tests/OptimizerTextTestRunner.hpp
index 7904ac3..27fa14f 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.hpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_TESTS_OPTIMIZER_TEXT_TEST_RUNNER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/TestDatabaseLoader.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/TestDatabaseLoader.cpp b/query_optimizer/tests/TestDatabaseLoader.cpp
index 764ff2f..f40a3e0 100644
--- a/query_optimizer/tests/TestDatabaseLoader.cpp
+++ b/query_optimizer/tests/TestDatabaseLoader.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/tests/TestDatabaseLoader.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/TestDatabaseLoader.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/TestDatabaseLoader.hpp b/query_optimizer/tests/TestDatabaseLoader.hpp
index 80dbfd2..d49719d 100644
--- a/query_optimizer/tests/TestDatabaseLoader.hpp
+++ b/query_optimizer/tests/TestDatabaseLoader.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_TESTS_TEST_DATABASE_LOADER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/CMakeLists.txt b/query_optimizer/tests/execution_generator/CMakeLists.txt
index 56bae16..1980980 100644
--- a/query_optimizer/tests/execution_generator/CMakeLists.txt
+++ b/query_optimizer/tests/execution_generator/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_test(quickstep_queryoptimizer_tests_executiongenerator_create
          "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Create.test b/query_optimizer/tests/execution_generator/Create.test
index 94da3d6..5bd0e76 100644
--- a/query_optimizer/tests/execution_generator/Create.test
+++ b/query_optimizer/tests/execution_generator/Create.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 CREATE TABLE FOO (col1 INT,
                   col2 LONG,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Delete.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Delete.test b/query_optimizer/tests/execution_generator/Delete.test
index 8c6864c..c641bcb 100644
--- a/query_optimizer/tests/execution_generator/Delete.test
+++ b/query_optimizer/tests/execution_generator/Delete.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 DELETE FROM test WHERE int_col+float_col > 0;
 SELECT int_col+float_col FROM test;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Distinct.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Distinct.test b/query_optimizer/tests/execution_generator/Distinct.test
index feab898..d8f5453 100644
--- a/query_optimizer/tests/execution_generator/Distinct.test
+++ b/query_optimizer/tests/execution_generator/Distinct.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 CREATE TABLE foo(x INT, y DOUBLE, z INT);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Drop.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Drop.test b/query_optimizer/tests/execution_generator/Drop.test
index 44fd0f1..4592758 100644
--- a/query_optimizer/tests/execution_generator/Drop.test
+++ b/query_optimizer/tests/execution_generator/Drop.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 DROP TABLE test;
 SELECT * FROM test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Index.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Index.test b/query_optimizer/tests/execution_generator/Index.test
index 749afcb..bbaa526 100644
--- a/query_optimizer/tests/execution_generator/Index.test
+++ b/query_optimizer/tests/execution_generator/Index.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 CREATE TABLE foo3 (col1 INT, col2 INT);
 CREATE INDEX idx1 ON foo3(col1) USING CSBTREE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Insert.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Insert.test b/query_optimizer/tests/execution_generator/Insert.test
index fd4cfde..152e2a3 100644
--- a/query_optimizer/tests/execution_generator/Insert.test
+++ b/query_optimizer/tests/execution_generator/Insert.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 INSERT INTO test VALUES (100, 2, 1.1, 1.3, 'foo');
 INSERT INTO test VALUES (100, 3, 1.2, 1.4, 'foofoo');

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Join.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Join.test b/query_optimizer/tests/execution_generator/Join.test
index 011d365..c26b915 100644
--- a/query_optimizer/tests/execution_generator/Join.test
+++ b/query_optimizer/tests/execution_generator/Join.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Prepare testing relations
 CREATE TABLE a(w INT, x LONG, y DOUBLE, z VARCHAR(16));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 6bada6c..7d662e9 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 SELECT *
 FROM test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/StringPatternMatching.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/StringPatternMatching.test b/query_optimizer/tests/execution_generator/StringPatternMatching.test
index 122deeb..1cd4818 100644
--- a/query_optimizer/tests/execution_generator/StringPatternMatching.test
+++ b/query_optimizer/tests/execution_generator/StringPatternMatching.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#   University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Initialize tables
 CREATE TABLE foo (name VARCHAR(24),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/TableGenerator.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/TableGenerator.test b/query_optimizer/tests/execution_generator/TableGenerator.test
index 375635e..961ebd7 100644
--- a/query_optimizer/tests/execution_generator/TableGenerator.test
+++ b/query_optimizer/tests/execution_generator/TableGenerator.test
@@ -1,16 +1,19 @@
-#   Copyright 2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Use generate_series
 SELECT * FROM generate_series(1, 10);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/execution_generator/Update.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Update.test b/query_optimizer/tests/execution_generator/Update.test
index 5b3709b..e2033dc 100644
--- a/query_optimizer/tests/execution_generator/Update.test
+++ b/query_optimizer/tests/execution_generator/Update.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Both assignments need coercion.
 UPDATE test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/logical_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/CMakeLists.txt b/query_optimizer/tests/logical_generator/CMakeLists.txt
index 4edddfa..58d3d77 100644
--- a/query_optimizer/tests/logical_generator/CMakeLists.txt
+++ b/query_optimizer/tests/logical_generator/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_test(quickstep_queryoptimizer_tests_logicalgenerator_create
          "../quickstep_queryoptimizer_tests_OptimizerTextTest"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/logical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Create.test b/query_optimizer/tests/logical_generator/Create.test
index eb99759..a35a92a 100644
--- a/query_optimizer/tests/logical_generator/Create.test
+++ b/query_optimizer/tests/logical_generator/Create.test
@@ -1,16 +1,19 @@
-#   Copyright 2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan]
 CREATE TABLE foo (attr int) WITH BLOCKPROPERTIES (TYPE rowstore, BLOCKSIZEMB 10)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/logical_generator/Index.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Index.test b/query_optimizer/tests/logical_generator/Index.test
index bbab354..727a69f 100644
--- a/query_optimizer/tests/logical_generator/Index.test
+++ b/query_optimizer/tests/logical_generator/Index.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan]
 CREATE INDEX csbIndex ON test(int_col) USING CSBTREE

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/logical_generator/Join.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Join.test b/query_optimizer/tests/logical_generator/Join.test
index a6452cc..44205aa 100644
--- a/query_optimizer/tests/logical_generator/Join.test
+++ b/query_optimizer/tests/logical_generator/Join.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan]
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/logical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Select.test b/query_optimizer/tests/logical_generator/Select.test
index e0003bf..4996110 100644
--- a/query_optimizer/tests/logical_generator/Select.test
+++ b/query_optimizer/tests/logical_generator/Select.test
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan]
 select * from test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/CMakeLists.txt b/query_optimizer/tests/physical_generator/CMakeLists.txt
index abc6d9b..c752cdd 100644
--- a/query_optimizer/tests/physical_generator/CMakeLists.txt
+++ b/query_optimizer/tests/physical_generator/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_test(quickstep_queryoptimizer_tests_physicalgenerator_copy
          "../quickstep_queryoptimizer_tests_OptimizerTextTest"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Copy.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Copy.test b/query_optimizer/tests/physical_generator/Copy.test
index a43997c..2f66415 100644
--- a/query_optimizer/tests/physical_generator/Copy.test
+++ b/query_optimizer/tests/physical_generator/Copy.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 copy test from 'test.txt'

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Create.test b/query_optimizer/tests/physical_generator/Create.test
index 8e6c64b..54af3fa 100644
--- a/query_optimizer/tests/physical_generator/Create.test
+++ b/query_optimizer/tests/physical_generator/Create.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 create table foo (col1 int not null,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Delete.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Delete.test b/query_optimizer/tests/physical_generator/Delete.test
index 5bf302b..3557282 100644
--- a/query_optimizer/tests/physical_generator/Delete.test
+++ b/query_optimizer/tests/physical_generator/Delete.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 delete from test where int_col+long_col>1-float_col and int_col<4

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Drop.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Drop.test b/query_optimizer/tests/physical_generator/Drop.test
index 4ff2161..841b867 100644
--- a/query_optimizer/tests/physical_generator/Drop.test
+++ b/query_optimizer/tests/physical_generator/Drop.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 drop table test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Index.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Index.test b/query_optimizer/tests/physical_generator/Index.test
index 2ef989a..39c73a6 100644
--- a/query_optimizer/tests/physical_generator/Index.test
+++ b/query_optimizer/tests/physical_generator/Index.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 CREATE INDEX csbIndex ON test(int_col) USING CSBTREE

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Insert.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Insert.test b/query_optimizer/tests/physical_generator/Insert.test
index 401fe9e..c06c708 100644
--- a/query_optimizer/tests/physical_generator/Insert.test
+++ b/query_optimizer/tests/physical_generator/Insert.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 insert into test values (1, 2, 1.1, 1.3, 'foo', 'foo')

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Join.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Join.test b/query_optimizer/tests/physical_generator/Join.test
index 45c9cbd..4ebbac7 100644
--- a/query_optimizer/tests/physical_generator/Join.test
+++ b/query_optimizer/tests/physical_generator/Join.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default physical_plan]
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index 76c5683..f81cad8 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 select * from test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/physical_generator/Update.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Update.test b/query_optimizer/tests/physical_generator/Update.test
index e753740..592c792 100644
--- a/query_optimizer/tests/physical_generator/Update.test
+++ b/query_optimizer/tests/physical_generator/Update.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default optimized_logical_plan physical_plan]
 update test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Aggregate.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Aggregate.test b/query_optimizer/tests/resolver/Aggregate.test
index 318378b..56c7c05 100644
--- a/query_optimizer/tests/resolver/Aggregate.test
+++ b/query_optimizer/tests/resolver/Aggregate.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # This file contains queries that test aggregation in SELECT.
 # Update.test, Delete.test contain other tests for aggregation in UPDATE and DELETE.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/CMakeLists.txt b/query_optimizer/tests/resolver/CMakeLists.txt
index 4d5a504..3101b43 100644
--- a/query_optimizer/tests/resolver/CMakeLists.txt
+++ b/query_optimizer/tests/resolver/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_test(quickstep_queryoptimizer_tests_resolver_aggregate
          "../quickstep_queryoptimizer_tests_OptimizerTextTest"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Copy.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Copy.test b/query_optimizer/tests/resolver/Copy.test
index 916efbc..c2ae91a 100644
--- a/query_optimizer/tests/resolver/Copy.test
+++ b/query_optimizer/tests/resolver/Copy.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default initial_logical_plan]
 copy test from 'test.txt'

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index 63f7ac9..7cd980f 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default initial_logical_plan]
 create table test (col int)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Delete.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Delete.test b/query_optimizer/tests/resolver/Delete.test
index 2729591..80b5a76 100644
--- a/query_optimizer/tests/resolver/Delete.test
+++ b/query_optimizer/tests/resolver/Delete.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default initial_logical_plan]
 delete from test where int_col+long_col>1-float_col and int_col<4

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Drop.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Drop.test b/query_optimizer/tests/resolver/Drop.test
index 6fc71d5..2c4fd99 100644
--- a/query_optimizer/tests/resolver/Drop.test
+++ b/query_optimizer/tests/resolver/Drop.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default initial_logical_plan]
 drop table test

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Index.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Index.test b/query_optimizer/tests/resolver/Index.test
index 1164172..3e383c5 100644
--- a/query_optimizer/tests/resolver/Index.test
+++ b/query_optimizer/tests/resolver/Index.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default initial_logical_plan]
 CREATE INDEX csbIndex ON test(int_col) USING CSBTREE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Insert.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Insert.test b/query_optimizer/tests/resolver/Insert.test
index a2ab47e..88fff53 100644
--- a/query_optimizer/tests/resolver/Insert.test
+++ b/query_optimizer/tests/resolver/Insert.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Literal values are coerced to the type of their corresponding columns.
 [default initial_logical_plan]



[15/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/IntervalParser.hpp
----------------------------------------------------------------------
diff --git a/types/IntervalParser.hpp b/types/IntervalParser.hpp
index 0b9d499..bafd6e1 100644
--- a/types/IntervalParser.hpp
+++ b/types/IntervalParser.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_INTERVAL_PARSER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/LongType.cpp
----------------------------------------------------------------------
diff --git a/types/LongType.cpp b/types/LongType.cpp
index feedb9d..fbf8d30 100644
--- a/types/LongType.cpp
+++ b/types/LongType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/LongType.hpp
----------------------------------------------------------------------
diff --git a/types/LongType.hpp b/types/LongType.hpp
index 0721035..a90dd32 100644
--- a/types/LongType.hpp
+++ b/types/LongType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_LONG_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/NullCoercibilityCheckMacro.hpp
----------------------------------------------------------------------
diff --git a/types/NullCoercibilityCheckMacro.hpp b/types/NullCoercibilityCheckMacro.hpp
index 59691e8..70e1beb 100644
--- a/types/NullCoercibilityCheckMacro.hpp
+++ b/types/NullCoercibilityCheckMacro.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_NULL_COERCIBILITY_CHECK_MACRO_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/NullType.hpp
----------------------------------------------------------------------
diff --git a/types/NullType.hpp b/types/NullType.hpp
index 8c15d49..c27a584 100644
--- a/types/NullType.hpp
+++ b/types/NullType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_NULLTYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/NumericSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/NumericSuperType.hpp b/types/NumericSuperType.hpp
index ec487e7..0cc1546 100644
--- a/types/NumericSuperType.hpp
+++ b/types/NumericSuperType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_NUMERIC_SUPER_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/NumericTypeUnifier.hpp
----------------------------------------------------------------------
diff --git a/types/NumericTypeUnifier.hpp b/types/NumericTypeUnifier.hpp
index 6ea8510..168dfb1 100644
--- a/types/NumericTypeUnifier.hpp
+++ b/types/NumericTypeUnifier.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_NUMERIC_TYPE_UNIFIER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/Type.cpp
----------------------------------------------------------------------
diff --git a/types/Type.cpp b/types/Type.cpp
index b89b2ac..f3d3f1b 100644
--- a/types/Type.cpp
+++ b/types/Type.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/Type.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/Type.hpp
----------------------------------------------------------------------
diff --git a/types/Type.hpp b/types/Type.hpp
index 84a0de0..0e8c4e5 100644
--- a/types/Type.hpp
+++ b/types/Type.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/Type.proto
----------------------------------------------------------------------
diff --git a/types/Type.proto b/types/Type.proto
index cfcb713..d03b5a4 100644
--- a/types/Type.proto
+++ b/types/Type.proto
@@ -1,17 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypeErrors.hpp
----------------------------------------------------------------------
diff --git a/types/TypeErrors.hpp b/types/TypeErrors.hpp
index 6348cdd..d547f9c 100644
--- a/types/TypeErrors.hpp
+++ b/types/TypeErrors.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_TYPE_ERRORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypeFactory.cpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.cpp b/types/TypeFactory.cpp
index 3709a50..7403dc9 100644
--- a/types/TypeFactory.cpp
+++ b/types/TypeFactory.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/TypeFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypeFactory.hpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.hpp b/types/TypeFactory.hpp
index e63a613..742348e 100644
--- a/types/TypeFactory.hpp
+++ b/types/TypeFactory.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_TYPE_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypeID.cpp
----------------------------------------------------------------------
diff --git a/types/TypeID.cpp b/types/TypeID.cpp
index 6a5a4c7..ff2f8e6 100644
--- a/types/TypeID.cpp
+++ b/types/TypeID.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/TypeID.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypeID.hpp
----------------------------------------------------------------------
diff --git a/types/TypeID.hpp b/types/TypeID.hpp
index 069e9c0..c54d8a5 100644
--- a/types/TypeID.hpp
+++ b/types/TypeID.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_TYPE_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypedValue.cpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.cpp b/types/TypedValue.cpp
index cef7cb0..8dd8b60 100644
--- a/types/TypedValue.cpp
+++ b/types/TypedValue.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/TypedValue.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypedValue.hpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.hpp b/types/TypedValue.hpp
index 10c3c05..d75720a 100644
--- a/types/TypedValue.hpp
+++ b/types/TypedValue.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_TYPED_VALUE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypedValue.proto
----------------------------------------------------------------------
diff --git a/types/TypedValue.proto b/types/TypedValue.proto
index c265c73..7f3ab7a 100644
--- a/types/TypedValue.proto
+++ b/types/TypedValue.proto
@@ -1,17 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/TypesModule.hpp
----------------------------------------------------------------------
diff --git a/types/TypesModule.hpp b/types/TypesModule.hpp
index b5cdd73..42316f7 100644
--- a/types/TypesModule.hpp
+++ b/types/TypesModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup Types

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 19dec28..02845b1 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/VarCharType.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index 3b46089..bb50e92 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_VAR_CHAR_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/YearMonthIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.cpp b/types/YearMonthIntervalType.cpp
index 10b29c4..3c15a91 100644
--- a/types/YearMonthIntervalType.cpp
+++ b/types/YearMonthIntervalType.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index 1ed7552..a2ba175 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_YEAR_MONTH_INTERVAL_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/containers/CMakeLists.txt b/types/containers/CMakeLists.txt
index 20c9e6c..aacb63a 100644
--- a/types/containers/CMakeLists.txt
+++ b/types/containers/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 QS_PROTOBUF_GENERATE_CPP(types_containers_Tuple_proto_srcs types_containers_Tuple_proto_hdrs
                          Tuple.proto)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/ColumnVector.cpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.cpp b/types/containers/ColumnVector.cpp
index 4fef94a..dfc0fae 100644
--- a/types/containers/ColumnVector.cpp
+++ b/types/containers/ColumnVector.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/containers/ColumnVector.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index 76968ba..fc65656 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_CONTAINERS_COLUMN_VECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/ColumnVectorUtil.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorUtil.hpp b/types/containers/ColumnVectorUtil.hpp
index 0ad5f6c..5a560a4 100644
--- a/types/containers/ColumnVectorUtil.hpp
+++ b/types/containers/ColumnVectorUtil.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_CONTAINERS_COLUMN_VECTOR_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index f1d29a2..2300f3b 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_CONTAINERS_COLUMN_VECTORS_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/Tuple.hpp
----------------------------------------------------------------------
diff --git a/types/containers/Tuple.hpp b/types/containers/Tuple.hpp
index 7e62c8f..60f832c 100644
--- a/types/containers/Tuple.hpp
+++ b/types/containers/Tuple.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_CONTAINERS_TUPLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/Tuple.proto
----------------------------------------------------------------------
diff --git a/types/containers/Tuple.proto b/types/containers/Tuple.proto
index d481a00..a605983 100644
--- a/types/containers/Tuple.proto
+++ b/types/containers/Tuple.proto
@@ -1,16 +1,19 @@
-//   Copyright 2015 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/containers/tests/ColumnVector_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/containers/tests/ColumnVector_unittest.cpp b/types/containers/tests/ColumnVector_unittest.cpp
index 6fb5caf..4cc676d 100644
--- a/types/containers/tests/ColumnVector_unittest.cpp
+++ b/types/containers/tests/ColumnVector_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/CMakeLists.txt b/types/operations/CMakeLists.txt
index ab0450f..c5dad0f 100644
--- a/types/operations/CMakeLists.txt
+++ b/types/operations/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_subdirectory(binary_operations)
 add_subdirectory(comparisons)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/Operation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/Operation.cpp b/types/operations/Operation.cpp
index d2fb96c..464955f 100644
--- a/types/operations/Operation.cpp
+++ b/types/operations/Operation.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/Operation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/Operation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/Operation.hpp b/types/operations/Operation.hpp
index 01567a0..51178b5 100644
--- a/types/operations/Operation.hpp
+++ b/types/operations/Operation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/Operation.proto
----------------------------------------------------------------------
diff --git a/types/operations/Operation.proto b/types/operations/Operation.proto
index 33e6b09..d6391f0 100644
--- a/types/operations/Operation.proto
+++ b/types/operations/Operation.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//     University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 
@@ -78,7 +78,7 @@ message SubstringOperation {
   extend UnaryOperation {
     // Required when operation_id = SUBSTRING.
     optional int64 start_position = 100;
-    optional int64 substring_length = 101;    
+    optional int64 substring_length = 101;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/AddBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AddBinaryOperation.cpp b/types/operations/binary_operations/AddBinaryOperation.cpp
index 6e6e839..2ee551b 100644
--- a/types/operations/binary_operations/AddBinaryOperation.cpp
+++ b/types/operations/binary_operations/AddBinaryOperation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/AddBinaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/AddBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AddBinaryOperation.hpp b/types/operations/binary_operations/AddBinaryOperation.hpp
index 6df4d7b..bc862bf 100644
--- a/types/operations/binary_operations/AddBinaryOperation.hpp
+++ b/types/operations/binary_operations/AddBinaryOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ADD_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperation.hpp b/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
index 3cad289..f9a27a8 100644
--- a/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
+++ b/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
index e5e1493..7224a0c 100644
--- a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
+++ b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/BinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.cpp b/types/operations/binary_operations/BinaryOperation.cpp
index 29a6217..436086f 100644
--- a/types/operations/binary_operations/BinaryOperation.cpp
+++ b/types/operations/binary_operations/BinaryOperation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/BinaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index e326528..585a1c6 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/BinaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationFactory.cpp b/types/operations/binary_operations/BinaryOperationFactory.cpp
index acd3d9e..51b5a7e 100644
--- a/types/operations/binary_operations/BinaryOperationFactory.cpp
+++ b/types/operations/binary_operations/BinaryOperationFactory.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/BinaryOperationFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/BinaryOperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationFactory.hpp b/types/operations/binary_operations/BinaryOperationFactory.hpp
index e10ccf4..578dfb8 100644
--- a/types/operations/binary_operations/BinaryOperationFactory.hpp
+++ b/types/operations/binary_operations/BinaryOperationFactory.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/BinaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationID.cpp b/types/operations/binary_operations/BinaryOperationID.cpp
index 22e2f77..7ba2e69 100644
--- a/types/operations/binary_operations/BinaryOperationID.cpp
+++ b/types/operations/binary_operations/BinaryOperationID.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/binary_operations/BinaryOperationID.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/binary_operations/BinaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationID.hpp b/types/operations/binary_operations/BinaryOperationID.hpp
index d3a57f4..2cf20d1 100644
--- a/types/operations/binary_operations/BinaryOperationID.hpp
+++ b/types/operations/binary_operations/BinaryOperationID.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_ID_HPP_



[07/50] [abbrv] incubator-quickstep git commit: Added TMB Client Id in QueryHandle.

Posted by zu...@apache.org.
Added TMB Client Id in QueryHandle.


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

Branch: refs/heads/travis-grpc
Commit: 5e5ec9e3bbe12015fdac950f552672562f1312bf
Parents: 1b07eaa
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Wed Aug 3 12:19:48 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Thu Aug 4 09:44:23 2016 -0700

----------------------------------------------------------------------
 cli/CommandExecutor.cpp                           |  7 ++++++-
 cli/QuickstepCli.cpp                              | 16 ++++++++++------
 cli/tests/CommandExecutorTestRunner.cpp           |  2 +-
 .../tests/QueryManagerSingleNode_unittest.cpp     |  2 +-
 query_optimizer/CMakeLists.txt                    |  5 ++---
 query_optimizer/QueryHandle.hpp                   | 18 +++++++++++++++++-
 query_optimizer/QueryProcessor.cpp                | 12 +++---------
 query_optimizer/QueryProcessor.hpp                | 14 +++++++++++---
 .../tests/ExecutionGeneratorTestRunner.cpp        |  2 +-
 9 files changed, 52 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 8acfae8..82c1ad9 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -210,9 +210,14 @@ inline TypedValue executeQueryForSingleResult(
   ParseResult result = parser_wrapper->getNextStatement();
   DCHECK(result.condition == ParseResult::kSuccess);
 
+  const ParseStatement &statement = *result.parsed_statement;
+
   // Generate the query plan.
   std::unique_ptr<QueryHandle> query_handle(
-      query_processor->generateQueryHandle(*result.parsed_statement));
+      std::make_unique<QueryHandle>(query_processor->query_id(),
+                                    main_thread_client_id,
+                                    statement.getPriority()));
+  query_processor->generateQueryHandle(statement, query_handle.get());
   DCHECK(query_handle->getQueryPlanMutable() != nullptr);
 
   // Use foreman to execute the query plan.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 154c689..78d7765 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -403,16 +403,17 @@ int main(int argc, char* argv[]) {
     bool reset_parser = false;
     for (;;) {
       ParseResult result = parser_wrapper->getNextStatement();
+      const ParseStatement &statement = *result.parsed_statement;
       if (result.condition == ParseResult::kSuccess) {
-        if (result.parsed_statement->getStatementType() == ParseStatement::kQuit) {
+        if (statement.getStatementType() == ParseStatement::kQuit) {
           quitting = true;
           break;
         }
 
-        if (result.parsed_statement->getStatementType() == ParseStatement::kCommand) {
+        if (statement.getStatementType() == ParseStatement::kCommand) {
           try {
             quickstep::cli::executeCommand(
-                *result.parsed_statement,
+                statement,
                 *(query_processor->getDefaultDatabase()),
                 main_thread_client_id,
                 foreman.getBusClientID(),
@@ -426,12 +427,15 @@ int main(int argc, char* argv[]) {
             reset_parser = true;
             break;
           }
-        continue;
+          continue;
         }
 
-        std::unique_ptr<QueryHandle> query_handle;
+        std::unique_ptr<QueryHandle> query_handle(
+            std::make_unique<QueryHandle>(query_processor->query_id(),
+                                          main_thread_client_id,
+                                          statement.getPriority()));
         try {
-          query_handle.reset(query_processor->generateQueryHandle(*result.parsed_statement));
+          query_processor->generateQueryHandle(statement, query_handle.get());
         } catch (const quickstep::SqlError &sql_error) {
           fprintf(stderr, "%s", sql_error.formatMessage(*command_string).c_str());
           reset_parser = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/cli/tests/CommandExecutorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.cpp b/cli/tests/CommandExecutorTestRunner.cpp
index dc7e43f..56f4ab9 100644
--- a/cli/tests/CommandExecutorTestRunner.cpp
+++ b/cli/tests/CommandExecutorTestRunner.cpp
@@ -98,7 +98,7 @@ void CommandExecutorTestRunner::runTestCase(
               nullptr,
               output_stream.file());
         } else  {
-          QueryHandle query_handle(0 /* query_id */);
+          QueryHandle query_handle(0 /* query_id */, main_thread_client_id_);
           O::LogicalGenerator logical_generator(&optimizer_context);
           O::PhysicalGenerator physical_generator;
           O::ExecutionGenerator execution_generator(&optimizer_context, &query_handle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 09ae6ba..050b315 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -232,7 +232,7 @@ class QueryManagerTest : public ::testing::Test {
     db_.reset(new CatalogDatabase(nullptr /* catalog */, "database"));
     storage_manager_.reset(new StorageManager("./"));
     bus_.Initialize();
-    query_handle_.reset(new QueryHandle(0));  // dummy query ID.
+    query_handle_.reset(new QueryHandle(0 /* dummy query ID */, tmb::kClientIdNone /* cli_id */));
     query_plan_ = query_handle_->getQueryPlanMutable();
     query_handle_->getQueryContextProtoMutable()->set_query_id(query_handle_->query_id());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 04e54d3..de63bd0 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -211,7 +211,8 @@ target_link_libraries(quickstep_queryoptimizer_QueryHandle
                       quickstep_catalog_Catalog_proto
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryoptimizer_QueryPlan
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_queryoptimizer_QueryPlan
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_utility_DAG
@@ -219,9 +220,7 @@ target_link_libraries(quickstep_queryoptimizer_QueryPlan
 target_link_libraries(quickstep_queryoptimizer_QueryProcessor
                       quickstep_catalog_Catalog
                       quickstep_catalog_Catalog_proto
-                      quickstep_parser_ParseStatement
                       quickstep_queryoptimizer_Optimizer
-                      quickstep_queryoptimizer_QueryHandle
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_Validator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/QueryHandle.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryHandle.hpp b/query_optimizer/QueryHandle.hpp
index 5f3649a..55427cd 100644
--- a/query_optimizer/QueryHandle.hpp
+++ b/query_optimizer/QueryHandle.hpp
@@ -20,13 +20,14 @@
 #include <cstddef>
 #include <cstdint>
 #include <memory>
-#include <utility>
 
 #include "catalog/Catalog.pb.h"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
 #include "utility/Macros.hpp"
 
+#include "tmb/id_typedefs.h"
+
 namespace quickstep {
 
 class CatalogRelation;
@@ -44,10 +45,14 @@ class QueryHandle {
    * @brief Constructor.
    *
    * @param query_id The given query id.
+   * @param cli_id The client id of the CLI which submits the query.
+   * @param query_priority The priority of this query.
    */
   explicit QueryHandle(const std::size_t query_id,
+                       const tmb::client_id cli_id,
                        const std::uint64_t query_priority = 1)
       : query_id_(query_id),
+        cli_id_(cli_id),
         query_priority_(query_priority),
         query_plan_(new QueryPlan()),
         query_result_relation_(nullptr) {}
@@ -64,6 +69,13 @@ class QueryHandle {
   }
 
   /**
+   * @brief Get the client id of the CLI which submits the query.
+   */
+  tmb::client_id getClientId() const {
+    return cli_id_;
+  }
+
+  /**
    * @brief Get the query priority.
    **/
   const std::uint64_t query_priority() const {
@@ -121,6 +133,10 @@ class QueryHandle {
 
  private:
   const std::size_t query_id_;
+
+  // The client id of the CLI which submits the query.
+  const tmb::client_id cli_id_;
+
   const std::uint64_t query_priority_;
 
   std::unique_ptr<QueryPlan> query_plan_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/QueryProcessor.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.cpp b/query_optimizer/QueryProcessor.cpp
index 8af4408..f3693a1 100644
--- a/query_optimizer/QueryProcessor.cpp
+++ b/query_optimizer/QueryProcessor.cpp
@@ -23,28 +23,22 @@
 
 #include "catalog/Catalog.hpp"
 #include "catalog/Catalog.pb.h"
-#include "parser/ParseStatement.hpp"
 #include "query_optimizer/Optimizer.hpp"
-#include "query_optimizer/QueryHandle.hpp"
 
 using std::ifstream;
 using std::ofstream;
 
 namespace quickstep {
 
-QueryHandle* QueryProcessor::generateQueryHandle(const ParseStatement &statement) {
-  std::unique_ptr<QueryHandle> query_handle(
-      new QueryHandle(query_id_, statement.getPriority()));
-
-  optimizer_->generateQueryHandle(statement, query_handle.get());
+void QueryProcessor::generateQueryHandle(const ParseStatement &statement,
+                                         QueryHandle *query_handle) {
+  optimizer_->generateQueryHandle(statement, query_handle);
 
   if (optimizer_->isCatalogChanged() && !catalog_altered_) {
     catalog_altered_ = true;
   }
 
   ++query_id_;
-
-  return query_handle.release();
 }
 
 void QueryProcessor::saveCatalog() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/QueryProcessor.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.hpp b/query_optimizer/QueryProcessor.hpp
index f3844a0..43e462f 100644
--- a/query_optimizer/QueryProcessor.hpp
+++ b/query_optimizer/QueryProcessor.hpp
@@ -151,13 +151,21 @@ class QueryProcessor {
   ~QueryProcessor() {}
 
   /**
-   * @brief Create a query handle for the given parsed SQL statement. This
+   * @brief Get the next query id.
+   **/
+  std::size_t query_id() const {
+    return query_id_;
+  }
+
+  /**
+   * @brief Fill a query handle for the given parsed SQL statement. This
    *        includes that the optimizer creates a QueryPlan inside the handle.
    *
    * @param statement The parsed SQL statement to generate a query handle for.
-   * @return A query handle for statement.
+   * @param query_handle The generated query handle to output.
    **/
-  QueryHandle* generateQueryHandle(const ParseStatement &statement);
+  void generateQueryHandle(const ParseStatement &statement,
+                           QueryHandle *query_handle);
 
   /**
    * @brief Save the catalog back to disk.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index fd1bb86..195ca59 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -84,7 +84,7 @@ void ExecutionGeneratorTestRunner::runTestCase(
     } else {
       std::printf("%s\n", result.parsed_statement->toString().c_str());
       try {
-        QueryHandle query_handle(0 /* query_id */);
+        QueryHandle query_handle(0 /* query_id */, main_thread_client_id_);
         LogicalGenerator logical_generator(&optimizer_context);
         PhysicalGenerator physical_generator;
         ExecutionGenerator execution_generator(&optimizer_context,


[03/50] [abbrv] incubator-quickstep git commit: Implemented hashjoin optimization class and removed the logic from ExecutionGenerator.

Posted by zu...@apache.org.
Implemented hashjoin optimization class and removed the logic from ExecutionGenerator.


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

Branch: refs/heads/travis-grpc
Commit: a61b99e9e1fcbbe84c60d63b8277cbb67e518030
Parents: ccea2ff
Author: Hakan Memisoglu <ha...@gmail.com>
Authored: Mon Aug 1 16:39:07 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Wed Aug 3 03:10:58 2016 -0500

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |  1 +
 query_optimizer/ExecutionGenerator.cpp          | 19 +-----
 query_optimizer/PhysicalGenerator.cpp           |  2 +
 query_optimizer/physical/HashJoin.hpp           |  1 +
 query_optimizer/rules/BottomUpRule.hpp          | 10 ++++
 query_optimizer/rules/CMakeLists.txt            | 12 ++++
 query_optimizer/rules/SwapProbeBuild.cpp        | 62 ++++++++++++++++++++
 query_optimizer/rules/SwapProbeBuild.hpp        | 48 +++++++++++++++
 .../tests/physical_generator/Select.test        | 36 ++++++------
 9 files changed, 156 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index a56b714..c55881f 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -199,6 +199,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
+                      quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate
                       quickstep_queryoptimizer_strategy_Join
                       quickstep_queryoptimizer_strategy_OneToOne

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 88103df..fb24489 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -604,6 +604,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const CatalogRelation *referenced_stored_probe_relation = nullptr;
   const CatalogRelation *referenced_stored_build_relation = nullptr;
 
+  std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
+
   bool any_probe_attributes_nullable = false;
   bool any_build_attributes_nullable = false;
 
@@ -671,23 +673,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     key_types.push_back(&left_attribute_type);
   }
 
-  std::size_t probe_cardinality = cost_model_->estimateCardinality(probe_physical);
-  std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
-  // For inner join, we may swap the probe table and the build table.
-  if (physical_plan->join_type() == P::HashJoin::JoinType::kInnerJoin)  {
-    // Choose the smaller table as the inner build table,
-    // and the other one as the outer probe table.
-    if (probe_cardinality < build_cardinality) {
-      // Switch the probe and build physical nodes.
-      std::swap(probe_physical, build_physical);
-      std::swap(probe_cardinality, build_cardinality);
-      std::swap(probe_attribute_ids, build_attribute_ids);
-      std::swap(any_probe_attributes_nullable, any_build_attributes_nullable);
-      std::swap(probe_original_attribute_ids, build_original_attribute_ids);
-      std::swap(referenced_stored_probe_relation, referenced_stored_build_relation);
-    }
-  }
-
   // Convert the residual predicate proto.
   QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
   if (physical_plan->residual_predicate()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 75a7bc9..897b212 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
+#include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
 #include "query_optimizer/strategy/Join.hpp"
 #include "query_optimizer/strategy/OneToOne.hpp"
@@ -98,6 +99,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
   }
   rules.emplace_back(new PruneColumns());
+  rules.emplace_back(new SwapProbeBuild());
 
   for (std::unique_ptr<Rule<P::Physical>> &rule : rules) {
     physical_plan_ = rule->apply(physical_plan_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index b904b5f..988b139 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_HASHJOIN_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_HASHJOIN_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <type_traits>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/BottomUpRule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/BottomUpRule.hpp b/query_optimizer/rules/BottomUpRule.hpp
index 0acc199..f98dadf 100644
--- a/query_optimizer/rules/BottomUpRule.hpp
+++ b/query_optimizer/rules/BottomUpRule.hpp
@@ -54,6 +54,7 @@ class BottomUpRule : public Rule<TreeType> {
   TreeNodePtr apply(const TreeNodePtr &tree) override {
     DCHECK(tree != nullptr);
 
+    init(tree);
     std::vector<std::shared_ptr<const TreeType>> new_children;
     bool has_changed_children = false;
     for (const std::shared_ptr<const TreeType> &child : tree->children()) {
@@ -80,6 +81,15 @@ class BottomUpRule : public Rule<TreeType> {
    */
   virtual TreeNodePtr applyToNode(const TreeNodePtr &node) = 0;
 
+  /**
+   * @brief Override this method to implement the initialization code
+   *        for the rule.
+   *
+   * @param input The input tree.
+   */
+  virtual void init(const TreeNodePtr &input) {
+  }
+
  private:
   DISALLOW_COPY_AND_ASSIGN(BottomUpRule);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 1990174..04a9814 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -29,6 +29,7 @@ add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
             StarSchemaHashJoinOrderOptimization.cpp
             StarSchemaHashJoinOrderOptimization.hpp)
+add_library(quickstep_queryoptimizer_rules_SwapProbeBuild SwapProbeBuild.cpp SwapProbeBuild.hpp)
 add_library(quickstep_queryoptimizer_rules_TopDownRule ../../empty_src.cpp TopDownRule.hpp)
 add_library(quickstep_queryoptimizer_rules_UpdateExpression UpdateExpression.cpp UpdateExpression.hpp)
 add_library(quickstep_queryoptimizer_rules_UnnestSubqueries UnnestSubqueries.cpp UnnestSubqueries.hpp)
@@ -127,6 +128,16 @@ target_link_libraries(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOpti
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_SwapProbeBuild
+                      quickstep_queryoptimizer_costmodel_SimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_BottomUpRule
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
@@ -185,6 +196,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
+                      quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_UpdateExpression
                       quickstep_queryoptimizer_rules_UnnestSubqueries)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/SwapProbeBuild.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/SwapProbeBuild.cpp b/query_optimizer/rules/SwapProbeBuild.cpp
new file mode 100644
index 0000000..4b4b38f
--- /dev/null
+++ b/query_optimizer/rules/SwapProbeBuild.cpp
@@ -0,0 +1,62 @@
+#include "query_optimizer/rules/SwapProbeBuild.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+
+
+namespace quickstep {
+namespace optimizer {
+
+P::PhysicalPtr SwapProbeBuild::applyToNode(const P::PhysicalPtr &input) {
+  P::HashJoinPtr hash_join;
+
+  if (P::SomeHashJoin::MatchesWithConditionalCast(input, &hash_join)
+      && hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
+    P::PhysicalPtr left = hash_join->left();
+    P::PhysicalPtr right = hash_join->right();
+
+    std::size_t left_cardinality = cost_model_->estimateCardinality(left);
+    std::size_t right_cardinality = cost_model_->estimateCardinality(right);
+
+    if (right_cardinality > left_cardinality) {
+      std::vector<E::AttributeReferencePtr> left_join_attributes = hash_join->left_join_attributes();
+      std::vector<E::AttributeReferencePtr> right_join_attributes = hash_join->right_join_attributes();
+
+      P::PhysicalPtr output = P::HashJoin::Create(right,
+                                                  left,
+                                                  right_join_attributes,
+                                                  left_join_attributes,
+                                                  hash_join->residual_predicate(),
+                                                  hash_join->project_expressions(),
+                                                  hash_join->join_type());
+      LOG_APPLYING_RULE(input, output);
+      return output;
+    }
+  }
+
+  LOG_IGNORING_RULE(input);
+  return input;
+}
+
+void SwapProbeBuild::init(const P::PhysicalPtr &input) {
+  if (cost_model_ == nullptr) {
+    P::TopLevelPlanPtr top_level;
+    if (P::SomeTopLevelPlan::MatchesWithConditionalCast(input, &top_level)) {
+      cost_model_.reset(new C::SimpleCostModel(top_level->shared_subplans()));
+    } else {
+      std::vector<P::PhysicalPtr> plans = {input};
+      cost_model_.reset(new C::SimpleCostModel(plans));
+    }
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/SwapProbeBuild.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/SwapProbeBuild.hpp b/query_optimizer/rules/SwapProbeBuild.hpp
new file mode 100644
index 0000000..da8fdf9
--- /dev/null
+++ b/query_optimizer/rules/SwapProbeBuild.hpp
@@ -0,0 +1,48 @@
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_SWAP_PROBE_BUILD_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_SWAP_PROBE_BUILD_HPP_
+
+#include <memory>
+#include <string>
+
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "query_optimizer/rules/BottomUpRule.hpp"
+#include "query_optimizer/cost_model/SimpleCostModel.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+namespace P = ::quickstep::optimizer::physical;
+namespace E = ::quickstep::optimizer::expressions;
+namespace C = ::quickstep::optimizer::cost;
+
+/**
+ * @brief Rule that applies to a physical plan to arrange probe and
+ *        build side based on the cardinalities.
+ */
+class SwapProbeBuild : public BottomUpRule<P::Physical> {
+ public:
+  SwapProbeBuild() {
+  }
+
+  std::string getName() const override { return "SwapProbeBuild"; }
+
+ protected:
+  P::PhysicalPtr applyToNode(const P::PhysicalPtr &input) override;
+  void init(const P::PhysicalPtr &input) override;
+
+ private:
+  std::unique_ptr<C::SimpleCostModel> cost_model_;
+
+  DISALLOW_COPY_AND_ASSIGN(SwapProbeBuild);
+};
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index d99916c..76c5683 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -2232,10 +2232,7 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=HashJoin
-| +-left=TableReference[relation=b]
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
-| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
-| +-right=Aggregate
+| +-left=Aggregate
 | | +-input=TableReference[relation=c]
 | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
@@ -2245,6 +2242,9 @@ TopLevelPlan
 | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
 | |     +-AggregateFunction[function=SUM]
 | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-right=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-project_expressions=
 | | +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
 | |   +-Add
@@ -2252,9 +2252,9 @@ TopLevelPlan
 | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
 | |       type=Long NULL]
 | +-left_join_attributes=
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | +-right_join_attributes=
-|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
 +-output_attributes=
   +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
     type=Long NULL]
@@ -2369,10 +2369,7 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=HashJoin
-| +-left=TableReference[relation=b]
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
-| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
-| +-right=Aggregate
+| +-left=Aggregate
 | | +-input=TableReference[relation=c]
 | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
@@ -2382,6 +2379,9 @@ TopLevelPlan
 | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
 | |     +-AggregateFunction[function=SUM]
 | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-right=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-residual_predicate=Greater
 | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | | +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
@@ -2390,9 +2390,9 @@ TopLevelPlan
 | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
 | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-left_join_attributes=
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | +-right_join_attributes=
-|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
 +-output_attributes=
   +-AttributeReference[id=0,name=w,relation=b,type=Int]
   +-AttributeReference[id=1,name=x,relation=b,type=Int]
@@ -2476,10 +2476,7 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=HashJoin
-| +-left=TableReference[relation=b]
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
-| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
-| +-right=NestedLoopsJoin
+| +-left=NestedLoopsJoin
 | | +-left=Aggregate
 | | | +-input=TableReference[relation=c]
 | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
@@ -2518,6 +2515,9 @@ TopLevelPlan
 | |       | type=Long NULL]
 | |       +-AttributeReference[id=9,name=,alias=$aggregate0,relation=$aggregate,
 | |         type=Long NULL]
+| +-right=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-project_expressions=
 | | +-Alias[id=11,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
 | |   +-Add
@@ -2525,9 +2525,9 @@ TopLevelPlan
 | |     +-AttributeReference[id=10,name=,alias=(SUM(y)+SubqueryExpression),
 | |       relation=,type=Long NULL]
 | +-left_join_attributes=
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | +-right_join_attributes=
-|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
 +-output_attributes=
   +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
     type=Long NULL]


[09/50] [abbrv] incubator-quickstep git commit: Minor changes in Shiftboss.

Posted by zu...@apache.org.
Minor changes in Shiftboss.


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

Branch: refs/heads/travis-grpc
Commit: bf7646f7ebf132e57c41cba93e611dea337dd8f0
Parents: 991f7a4
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Fri Aug 5 11:15:09 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Fri Aug 5 11:15:09 2016 -0700

----------------------------------------------------------------------
 query_execution/CMakeLists.txt              |  1 -
 query_execution/QueryManagerDistributed.cpp |  6 +++---
 query_execution/QueryManagerDistributed.hpp |  6 +++---
 query_execution/Shiftboss.cpp               | 11 +----------
 storage/StorageManager.hpp                  |  1 -
 5 files changed, 7 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf7646f7/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 8bf1ab1..edbe5d0 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -231,7 +231,6 @@ if (ENABLE_DISTRIBUTED)
                         quickstep_relationaloperators_WorkOrderFactory
                         quickstep_storage_InsertDestination
                         quickstep_storage_StorageBlock
-                        quickstep_storage_StorageBlockInfo
                         quickstep_storage_StorageManager
                         quickstep_threading_Thread
                         quickstep_threading_ThreadUtil

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf7646f7/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 5d589d5..9e192eb 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -42,11 +42,11 @@ using std::unique_ptr;
 namespace quickstep {
 
 QueryManagerDistributed::QueryManagerDistributed(QueryHandle *query_handle,
-                                                 ShiftbossDirectory *shiftbosses,
+                                                 ShiftbossDirectory *shiftboss_directory,
                                                  const tmb::client_id foreman_client_id,
                                                  tmb::MessageBus *bus)
     : QueryManagerBase(query_handle),
-      shiftbosses_(shiftbosses),
+      shiftboss_directory_(shiftboss_directory),
       foreman_client_id_(foreman_client_id),
       bus_(bus),
       normal_workorder_protos_container_(
@@ -168,7 +168,7 @@ bool QueryManagerDistributed::initiateRebuild(const dag_node_index index) {
   // TODO(zuyu): Multiple workers support.
   QueryExecutionUtil::SendTMBMessage(bus_,
                                      foreman_client_id_,
-                                     shiftbosses_->getClientId(0),
+                                     shiftboss_directory_->getClientId(0),
                                      move(tagged_msg));
 
   // The negative value indicates that the number of rebuild work orders is to be

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf7646f7/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 5d9be81..3eee95c 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -48,12 +48,12 @@ class QueryManagerDistributed final : public QueryManagerBase {
    * @brief Constructor.
    *
    * @param query_handle The QueryHandle object for this query.
-   * @param shiftbosses The ShiftbossDirectory to use.
+   * @param shiftboss_directory The ShiftbossDirectory to use.
    * @param foreman_client_id The TMB client ID of the foreman thread.
    * @param bus The TMB used for communication.
    **/
   QueryManagerDistributed(QueryHandle *query_handle,
-                          ShiftbossDirectory *shiftbosses,
+                          ShiftbossDirectory *shiftboss_directory,
                           const tmb::client_id foreman_client_id,
                           tmb::MessageBus *bus);
 
@@ -100,7 +100,7 @@ class QueryManagerDistributed final : public QueryManagerBase {
            (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
   }
 
-  ShiftbossDirectory *shiftbosses_;
+  ShiftbossDirectory *shiftboss_directory_;
 
   const tmb::client_id foreman_client_id_;
   tmb::MessageBus *bus_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf7646f7/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 3f7129b..50381b2 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -32,7 +32,6 @@
 #include "relational_operators/WorkOrderFactory.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 #include "threading/ThreadUtil.hpp"
 
@@ -156,15 +155,7 @@ void Shiftboss::run() {
         CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 
         for (int i = 0; i < proto.blocks_size(); ++i) {
-          const block_id block = proto.blocks(i);
-          storage_manager_->saveBlockOrBlob(block);
-          if (storage_manager_->blockOrBlobIsLoaded(block)) {
-            // NOTE(zuyu): eviction is required to avoid accesses to the query
-            // result relation schema in CatalogDatabaseCache, for all query
-            // optimizer execution generator unit tests and the single-process
-            // Quickstep CLI.
-            storage_manager_->evictBlockOrBlob(block);
-          }
+          storage_manager_->saveBlockOrBlob(proto.blocks(i));
         }
 
         serialization::SaveQueryResultResponseMessage proto_response;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf7646f7/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index 348018f..50ddb0f 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -619,7 +619,6 @@ class StorageManager {
   FRIEND_TEST(BlockLocatorTest, BlockTest);
   FRIEND_TEST(BlockLocatorTest, BlobTest);
 
-  friend class Shiftboss;
   FRIEND_TEST(StorageManagerTest, DifferentNUMANodeBlobTestWithEviction);
   FRIEND_TEST(StorageManagerTest, EvictFromSameShardTest);
 


[04/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-38. Add support for python3 to utility scripts

Posted by zu...@apache.org.
QUICKSTEP-38. Add support for python3 to utility scripts


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

Branch: refs/heads/travis-grpc
Commit: d4e714ce32e195c95a4e603db7ec8302865f9418
Parents: a61b99e
Author: Caleb Welton <cw...@apache.org>
Authored: Tue Aug 2 10:44:41 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Wed Aug 3 11:48:27 2016 -0700

----------------------------------------------------------------------
 cyclic_dependency.py   | 28 ++++++++++++++++------------
 validate_cmakelists.py | 42 +++++++++++++++++++++++-------------------
 2 files changed, 39 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d4e714ce/cyclic_dependency.py
----------------------------------------------------------------------
diff --git a/cyclic_dependency.py b/cyclic_dependency.py
index 8bf7d80..a5cca25 100755
--- a/cyclic_dependency.py
+++ b/cyclic_dependency.py
@@ -1,4 +1,4 @@
-#!/usr/bin/env python2
+#!/usr/bin/env python
 
 # Script to do analyze the dependencies in Quickstep particularly cycles in the
 # dependency graph. This script can be used to find:
@@ -33,6 +33,10 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
+from __future__ import absolute_import
+from __future__ import division
+from __future__ import print_function
+
 import itertools
 import networkx as nx
 from optparse import OptionParser
@@ -110,18 +114,18 @@ def process_cmakelists_file(cmakelists_filename, qs_module_dirs):
 # target mapping, and target to node mapping.
 def create_graph(deps_in_cmake):
     nodes = set()
-    for source, dest_set in deps_in_cmake.iteritems():
+    for source, dest_set in iter(deps_in_cmake.items()):
         nodes.add(source)
         nodes.update(dest_set)
 
     nodes_list = list(nodes)
     nodes_map = {}
-    for i, n in zip(xrange(len(nodes_list)), nodes_list):
+    for i, n in zip(range(len(nodes_list)), nodes_list):
         nodes_map[n] = i
 
     G = nx.DiGraph()
 
-    for source, dest_set in deps_in_cmake.iteritems():
+    for source, dest_set in iter(deps_in_cmake.items()):
         source_node = nodes_map[source]
         for dest in dest_set:
             if source == dest: continue
@@ -137,17 +141,17 @@ def find_strongly_connected_components(G, nodes_list):
         if len(n) > 1:
             components += 1
             # Only output components bigger than 1.
-            print [nodes_list[i] for i in n]
+            print([nodes_list[i] for i in n])
     return components
 
 # Lists cycles in the graph truncating to 100 cycles.
 def find_cycles(G, nodes_list, truncate):
     cycles = 0
     for n in nx.simple_cycles(G):
-        print [nodes_list[i] for i in n]
+        print([nodes_list[i] for i in n])
         cycles += 1
         if cycles >= truncate:
-            print "Many cycles found. Truncating to {0} cycles.".format(truncate)
+            print("Many cycles found. Truncating to {0} cycles.".format(truncate))
             break
     return cycles
 
@@ -156,16 +160,16 @@ def find_path(G, nodes_list, nodes_map, source, target):
     source_node = nodes_map[source]
     target_node = nodes_map[target]
     if nx.has_path(G, source_node, target_node):
-        print [nodes_list[i] for i in nx.shortest_path(G,
+        print([nodes_list[i] for i in nx.shortest_path(G,
                 source_node,
-                target_node)]
+                target_node)])
     else:
-        print 'No path.'
+        print('No path.')
 
 def main():
     if not os.getcwd().endswith("quickstep"):
-        print ("WARNING: you don't appear to be running in the root quickstep "
-               "source directory. Don't blame me if something goes wrong.")
+        print("WARNING: you don't appear to be running in the root quickstep "
+              "source directory. Don't blame me if something goes wrong.")
     qs_module_dirs = []
     for filename in os.listdir("."):
         if (os.path.isdir(filename)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d4e714ce/validate_cmakelists.py
----------------------------------------------------------------------
diff --git a/validate_cmakelists.py b/validate_cmakelists.py
index 7dd6fc5..cf25d28 100755
--- a/validate_cmakelists.py
+++ b/validate_cmakelists.py
@@ -1,4 +1,4 @@
-#!/usr/bin/env python2
+#!/usr/bin/env python
 
 """Script to do basic sanity checking for target_link_libraries() commands in
 CMakeLists.txt files.
@@ -31,6 +31,10 @@ TODO List / Known Issues & Limitations:
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
+from __future__ import absolute_import
+from __future__ import division
+from __future__ import print_function
+
 import os
 import sys
 
@@ -334,8 +338,8 @@ def process_cmakelists_file(cmakelists_filename, qs_module_dirs):
                 if "CMAKE_VALIDATE_IGNORE_END" in line:
                     scan_state = previous_state
                 elif "CMAKE_VALIDATE_IGNORE_BEGIN" in line:
-                    print "Nested IGNORE_BEGIN directives found in: "\
-                        + cmakelists_filename + ", exiting"
+                    print("Nested IGNORE_BEGIN directives found in: "
+                        + cmakelists_filename + ", exiting")
                     exit(-1)
                 else:
                     continue
@@ -397,25 +401,25 @@ def process_cmakelists_file(cmakelists_filename, qs_module_dirs):
                     stitched_string = ""
                     scan_state = CMAKE_SCANNING_NONE
     # After scanning, report any missing dependencies.
-    for target, include_deps in deps_from_includes.iteritems():
+    for target, include_deps in iter(deps_from_includes.items()):
         if target in skipped_targets:
             pass
         elif len(include_deps) != 0:
             if target not in deps_in_cmake:
                 if not (target in include_deps and len(include_deps) == 1):
                     validation_failed_targets.add(target)
-                    print "Missing target_link_libraries() for " + target + ":"
+                    print("Missing target_link_libraries() for " + target + ":")
                     for dep in sorted(include_deps):
-                        print "\t" + dep
+                        print("\t" + dep)
             else:
                 missing_deps = (include_deps
                                 - deps_in_cmake[target]
                                 - IGNORED_DEPENDENCIES)
                 if len(missing_deps) != 0:
                     validation_failed_targets.add(target)
-                    print "Missing target_link_libraries() for " + target + ":"
+                    print("Missing target_link_libraries() for " + target + ":")
                     for dep in sorted(missing_deps):
-                        print "\t" + dep
+                        print("\t" + dep)
         elif target == module_targetname:
             # Special case hack for module all-in-one library
             missing_deps = (frozenset(deps_from_includes.keys())
@@ -427,21 +431,21 @@ def process_cmakelists_file(cmakelists_filename, qs_module_dirs):
                     true_missing_deps.add(dep)
             if len(true_missing_deps) != 0:
                 validation_failed_targets.add(target)
-                print "Missing target_link_libraries() for " + target + ":"
+                print("Missing target_link_libraries() for " + target + ":")
                 for dep in sorted(true_missing_deps):
-                    print "\t" + dep
+                    print("\t" + dep)
     # Also report possibly superfluous extra dependencies.
-    for target, cmake_deps in deps_in_cmake.iteritems():
+    for target, cmake_deps in iter(deps_in_cmake.items()):
         if (target not in skipped_targets) and (target in deps_from_includes):
             extra_deps = cmake_deps - deps_from_includes[target]
             if target in extra_deps:
                 extra_deps.remove(target)
             if len(extra_deps) != 0 and target != module_targetname:
                 validation_failed_targets.add(target)
-                print ("Possibly superfluous target_link_libraries() for "
+                print("Possibly superfluous target_link_libraries() for "
                        + target + ":")
                 for dep in sorted(extra_deps):
-                    print "\t" + dep
+                    print("\t" + dep)
     return (validation_failed_targets, skipped_targets, generated_targets)
 
 def main(cmakelists_to_process):
@@ -461,8 +465,8 @@ def main(cmakelists_to_process):
             missing or superfluous dependencies.
     """
     if not os.getcwd().endswith("quickstep"):
-        print ("WARNING: you don't appear to be running in the root quickstep "
-               "source directory. Don't blame me if something goes wrong.")
+        print("WARNING: you don't appear to be running in the root quickstep "
+              "source directory. Don't blame me if something goes wrong.")
     qs_module_dirs = []
     for filename in os.listdir("."):
         if (os.path.isdir(filename)
@@ -493,17 +497,17 @@ def main(cmakelists_to_process):
         global_skipped_targets.update(local_skipped_targets)
         global_generated_targets.update(local_generated_targets)
     if len(global_skipped_targets) != 0:
-        print ("WARNING: The following targets had multiple add_library() "
+        print("WARNING: The following targets had multiple add_library() "
                + "commands and were NOT checked by this script (they should "
                + "be manually checked):")
         for target in sorted(global_skipped_targets):
-            print "\t" + target
+            print("\t" + target)
     if len(global_generated_targets) != 0:
-        print ("INFO: The add_library() commands for the following targets "
+        print("INFO: The add_library() commands for the following targets "
                + "appear to reference generated sources, so they were not "
                + "checked):")
         for target in sorted(global_generated_targets):
-            print "\t" + target
+            print("\t" + target)
     return len(global_validation_failed_targets)
 
 if __name__ == "__main__":


[06/50] [abbrv] incubator-quickstep git commit: Add visualization for execution plan DAGs combined with profiling stats

Posted by zu...@apache.org.
Add visualization for execution plan DAGs combined with profiling stats


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

Branch: refs/heads/travis-grpc
Commit: 1b07eaae6f3a1b591960a331190dd4d7634426bf
Parents: 8cd5a56
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Tue Aug 2 16:57:47 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Aug 3 17:37:31 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt                                  |   1 +
 cli/QuickstepCli.cpp                            |  19 +-
 query_execution/ForemanSingleNode.cpp           |  16 +-
 query_execution/ForemanSingleNode.hpp           |  11 +
 query_execution/PolicyEnforcerBase.cpp          |  16 +-
 query_execution/PolicyEnforcerBase.hpp          |  15 +-
 query_execution/QueryExecutionMessages.proto    |  10 +-
 query_execution/QueryExecutionTypedefs.hpp      |  15 ++
 query_execution/Worker.cpp                      |  15 +-
 .../tests/QueryManagerSingleNode_unittest.cpp   |   5 +
 relational_operators/AggregationOperator.hpp    |  11 +
 relational_operators/BuildHashOperator.hpp      |  13 ++
 relational_operators/CreateIndexOperator.hpp    |   4 +
 relational_operators/CreateTableOperator.hpp    |   5 +
 relational_operators/DeleteOperator.hpp         |   5 +
 relational_operators/DestroyHashOperator.hpp    |   6 +
 relational_operators/DropTableOperator.hpp      |   5 +
 .../FinalizeAggregationOperator.hpp             |   5 +
 relational_operators/HashJoinOperator.hpp       |  24 ++
 relational_operators/InsertOperator.hpp         |   5 +
 .../NestedLoopsJoinOperator.hpp                 |   5 +
 relational_operators/RelationalOperator.hpp     |  17 ++
 relational_operators/SampleOperator.hpp         |   5 +
 relational_operators/SaveBlocksOperator.hpp     |   5 +
 relational_operators/SelectOperator.hpp         |   9 +
 relational_operators/SortMergeRunOperator.hpp   |   5 +
 .../SortRunGenerationOperator.hpp               |   5 +
 relational_operators/TableGeneratorOperator.hpp |   5 +
 relational_operators/TextScanOperator.hpp       |   4 +
 relational_operators/UpdateOperator.hpp         |   5 +
 .../WindowAggregationOperator.hpp               |   5 +
 utility/CMakeLists.txt                          |  14 ++
 utility/ExecutionDAGVisualizer.cpp              | 230 +++++++++++++++++++
 utility/ExecutionDAGVisualizer.hpp              | 112 +++++++++
 34 files changed, 600 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 0bbde61..3192713 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -770,6 +770,7 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_queryoptimizer_QueryProcessor
                       quickstep_storage_PreloaderThread
                       quickstep_threading_ThreadIDBasedMap
+                      quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector
                       quickstep_utility_SqlError

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 68a3599..154c689 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -75,6 +75,7 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 
 #include "storage/PreloaderThread.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
+#include "utility/ExecutionDAGVisualizer.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 #include "utility/SqlError.hpp"
@@ -185,6 +186,10 @@ DEFINE_string(profile_file_name, "",
               // To put things in perspective, the first run is, in my experiments, about 5-10
               // times more expensive than the average run. That means the query needs to be
               // run at least a hundred times to make the impact of the first run small (< 5 %).
+DEFINE_bool(visualize_execution_dag, false,
+            "If true, visualize the execution plan DAG into a graph in DOT "
+            "format (DOT is a plain text graph description language) which is "
+            "then printed via stderr.");
 
 }  // namespace quickstep
 
@@ -361,7 +366,7 @@ int main(int argc, char* argv[]) {
       query_processor->getStorageManager(),
       -1,  // Don't pin the Foreman thread.
       num_numa_nodes_system,
-      quickstep::FLAGS_profile_and_report_workorder_perf);
+      quickstep::FLAGS_profile_and_report_workorder_perf || quickstep::FLAGS_visualize_execution_dag);
 
   // Start the worker threads.
   for (Worker &worker : workers) {
@@ -434,6 +439,12 @@ int main(int argc, char* argv[]) {
         }
 
         DCHECK(query_handle->getQueryPlanMutable() != nullptr);
+        std::unique_ptr<quickstep::ExecutionDAGVisualizer> dag_visualizer;
+        if (quickstep::FLAGS_visualize_execution_dag) {
+          dag_visualizer.reset(
+              new quickstep::ExecutionDAGVisualizer(*query_handle->getQueryPlanMutable()));
+        }
+
         start = std::chrono::steady_clock::now();
         QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
             main_thread_client_id,
@@ -471,6 +482,12 @@ int main(int argc, char* argv[]) {
             foreman.printWorkOrderProfilingResults(query_handle->query_id(),
                                                    stdout);
           }
+          if (quickstep::FLAGS_visualize_execution_dag) {
+            const auto &profiling_stats =
+                foreman.getWorkOrderProfilingResults(query_handle->query_id());
+            dag_visualizer->bindProfilingStats(profiling_stats);
+            std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
+          }
         } catch (const std::exception &e) {
           fprintf(stderr, "QUERY EXECUTION ERROR: %s\n", e.what());
           break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index d2b56ae..f935a0b 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -236,22 +236,26 @@ void ForemanSingleNode::sendWorkerMessage(const size_t worker_thread_index,
       << worker_directory_->getClientID(worker_thread_index);
 }
 
+const std::vector<WorkOrderTimeEntry>& ForemanSingleNode
+    ::getWorkOrderProfilingResults(const std::size_t query_id) const {
+  return policy_enforcer_->getProfilingResults(query_id);
+}
+
 void ForemanSingleNode::printWorkOrderProfilingResults(const std::size_t query_id,
                                                        std::FILE *out) const {
-  const std::vector<
-      std::tuple<std::size_t, std::size_t, std::size_t>>
-      &recorded_times = policy_enforcer_->getProfilingResults(query_id);
+  const std::vector<WorkOrderTimeEntry> &recorded_times =
+      policy_enforcer_->getProfilingResults(query_id);
   fputs("Query ID,Worker ID,NUMA Socket,Operator ID,Time (microseconds)\n", out);
   for (auto workorder_entry : recorded_times) {
     // Note: Index of the "worker thread index" in the tuple is 0.
-    const std::size_t worker_id = std::get<0>(workorder_entry);
+    const std::size_t worker_id = workorder_entry.worker_id;
     fprintf(out,
             "%lu,%lu,%d,%lu,%lu\n",
             query_id,
             worker_id,
             worker_directory_->getNUMANode(worker_id),
-            std::get<1>(workorder_entry),  // Operator ID.
-            std::get<2>(workorder_entry));  // Time.
+            workorder_entry.operator_id,  // Operator ID.
+            workorder_entry.end_time - workorder_entry.start_time);  // Time.
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/ForemanSingleNode.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.hpp b/query_execution/ForemanSingleNode.hpp
index caef5e0..d999095 100644
--- a/query_execution/ForemanSingleNode.hpp
+++ b/query_execution/ForemanSingleNode.hpp
@@ -76,6 +76,17 @@ class ForemanSingleNode final : public ForemanBase {
 
   ~ForemanSingleNode() override {}
 
+
+  /**
+   * @brief Get the results of profiling individual work orders for a given
+   *        query.
+   *
+   * @param query_id The ID of the query for which the results are to be printed.
+   * @return A vector of tuples, each being a single profiling entry.
+   **/
+  const std::vector<WorkOrderTimeEntry>& getWorkOrderProfilingResults(
+      const std::size_t query_id) const;
+
   /**
    * @brief Print the results of profiling individual work orders for a given
    *        query.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index d16a502..3371d6d 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -28,6 +28,7 @@
 #include "catalog/PartitionScheme.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryManagerBase.hpp"
 #include "relational_operators/WorkOrder.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -165,13 +166,14 @@ bool PolicyEnforcerBase::admitQueries(
 void PolicyEnforcerBase::recordTimeForWorkOrder(
     const serialization::NormalWorkOrderCompletionMessage &proto) {
   const std::size_t query_id = proto.query_id();
-  if (workorder_time_recorder_.find(query_id) == workorder_time_recorder_.end()) {
-    workorder_time_recorder_[query_id];
-  }
-  workorder_time_recorder_[query_id].emplace_back(
-      proto.worker_thread_index(),
-      proto.operator_index(),
-      proto.execution_time_in_microseconds());
+  std::vector<WorkOrderTimeEntry> &workorder_time_entries
+      = workorder_time_recorder_[query_id];
+  workorder_time_entries.emplace_back();
+  WorkOrderTimeEntry &entry = workorder_time_entries.back();
+  entry.worker_id = proto.worker_thread_index(),
+  entry.operator_id = proto.operator_index(),
+  entry.start_time = proto.execution_start_time(),
+  entry.end_time = proto.execution_end_time();
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/PolicyEnforcerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.hpp b/query_execution/PolicyEnforcerBase.hpp
index 0482ebc..15bc118 100644
--- a/query_execution/PolicyEnforcerBase.hpp
+++ b/query_execution/PolicyEnforcerBase.hpp
@@ -126,8 +126,8 @@ class PolicyEnforcerBase {
    *
    * @return A vector of tuples, each being a single profiling entry.
    **/
-  inline const std::vector<std::tuple<std::size_t, std::size_t, std::size_t>>&
-      getProfilingResults(const std::size_t query_id) const {
+  inline const std::vector<WorkOrderTimeEntry>& getProfilingResults(
+      const std::size_t query_id) const {
     DCHECK(profile_individual_workorders_);
     DCHECK(workorder_time_recorder_.find(query_id) !=
            workorder_time_recorder_.end());
@@ -158,16 +158,7 @@ class PolicyEnforcerBase {
   // The queries which haven't been admitted yet.
   std::queue<QueryHandle*> waiting_queries_;
 
-  // Key = Query ID.
-  // Value = A tuple indicating a record of executing a work order.
-  // Within a tuple ...
-  // 1st element: Logical worker ID.
-  // 2nd element: Operator ID.
-  // 3rd element: Time in microseconds to execute the work order.
-  std::unordered_map<
-      std::size_t,
-      std::vector<std::tuple<std::size_t, std::size_t, std::size_t>>>
-      workorder_time_recorder_;
+  WorkOrderTimeRecorder workorder_time_recorder_;
 
  private:
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f2219f6..5a089d2 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -38,7 +38,10 @@ message NormalWorkOrderCompletionMessage {
   required uint64 operator_index = 1;
   required uint64 worker_thread_index = 2;
   required uint64 query_id = 3;
-  optional uint64 execution_time_in_microseconds = 4;
+  
+  // Epoch time in microseconds.  
+  optional uint64 execution_start_time = 4;
+  optional uint64 execution_end_time = 5;
 }
 
 // A message sent upon completion of a rebuild WorkOrder execution.
@@ -46,7 +49,10 @@ message RebuildWorkOrderCompletionMessage {
   required uint64 operator_index = 1;
   required uint64 worker_thread_index = 2;
   required uint64 query_id = 3;
-  optional uint64 execution_time_in_microseconds = 4;
+
+  // Epoch time in microseconds.
+  optional uint64 execution_start_time = 4;
+  optional uint64 execution_end_time = 5;
 }
 
 message CatalogRelationNewBlockMessage {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index b67209f..4bbab59 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -18,6 +18,9 @@
 #ifndef QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_TYPEDEFS_HPP_
 #define QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_TYPEDEFS_HPP_
 
+#include <unordered_map>
+#include <vector>
+
 #include "query_optimizer/QueryOptimizerConfig.h"  // For QUICKSTEP_DISTRIBUTED
 #include "threading/ThreadIDBasedMap.hpp"
 
@@ -98,6 +101,18 @@ enum QueryExecutionMessageType : message_type_id {
 #endif
 };
 
+// WorkOrder profiling data structures.
+// Profiling record for an individual work order.
+struct WorkOrderTimeEntry {
+  std::size_t worker_id;
+  std::size_t operator_id;
+  std::size_t start_time;  // Epoch time measured in microseconds
+  std::size_t end_time;  // Epoch time measured in microseconds
+};
+// Key = query ID.
+// Value = vector of work order profiling records.
+typedef std::unordered_map<std::size_t, std::vector<WorkOrderTimeEntry>> WorkOrderTimeRecorder;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 6ba27f1..a582132 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -120,14 +120,21 @@ void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
   worker_message.getWorkOrder()->execute();
   end = std::chrono::steady_clock::now();
   delete worker_message.getWorkOrder();
-  const uint64_t execution_time_microseconds =
-      std::chrono::duration_cast<std::chrono::microseconds>(end - start)
-          .count();
+
+  // Convert the measured timestamps to epoch times in microseconds.
+  const uint64_t execution_start_time =
+      std::chrono::duration_cast<std::chrono::microseconds>(
+          start.time_since_epoch()).count();
+  const uint64_t execution_end_time =
+      std::chrono::duration_cast<std::chrono::microseconds>(
+          end.time_since_epoch()).count();
+
   // Construct the proto message.
   proto->set_operator_index(worker_message.getRelationalOpIndex());
   proto->set_query_id(query_id_for_workorder);
   proto->set_worker_thread_index(worker_thread_index_);
-  proto->set_execution_time_in_microseconds(execution_time_microseconds);
+  proto->set_execution_start_time(execution_start_time);
+  proto->set_execution_end_time(execution_end_time);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 39ca58c..09ae6ba 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -17,6 +17,7 @@
 
 #include <climits>
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -104,6 +105,10 @@ class MockOperator: public RelationalOperator {
         num_calls_donefeedingblocks_(0) {
   }
 
+  std::string getName() const override {
+    return "MockOperator";
+  }
+
 #define MOCK_OP_LOG(x) VLOG(x) << "Op[" << op_index_ << "]: " << __func__ << ": "
 
   // The methods below are used to check whether QueryManager calls the Relational

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 4bcbcf6..5bbf2f9 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATOR_HPP_
 
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -68,6 +69,7 @@ class AggregationOperator : public RelationalOperator {
                       bool input_relation_is_stored,
                       const QueryContext::aggregation_state_id aggr_state_index)
       : RelationalOperator(query_id),
+        input_relation_(input_relation),
         input_relation_is_stored_(input_relation_is_stored),
         input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
                                                            : std::vector<block_id>()),
@@ -77,6 +79,14 @@ class AggregationOperator : public RelationalOperator {
 
   ~AggregationOperator() override {}
 
+  std::string getName() const override {
+    return "AggregationOperator";
+  }
+
+  const CatalogRelation& input_relation() const {
+    return input_relation_;
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,
@@ -103,6 +113,7 @@ class AggregationOperator : public RelationalOperator {
    **/
   serialization::WorkOrder* createWorkOrderProto(const block_id block);
 
+  const CatalogRelation &input_relation_;
   const bool input_relation_is_stored_;
   std::vector<block_id> input_relation_block_ids_;
   const QueryContext::aggregation_state_id aggr_state_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 464bbf8..41346c8 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_BUILD_HASH_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_BUILD_HASH_OPERATOR_HPP_
 
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -93,6 +94,14 @@ class BuildHashOperator : public RelationalOperator {
 
   ~BuildHashOperator() override {}
 
+  const CatalogRelation& input_relation() const {
+    return input_relation_;
+  }
+
+  std::string getName() const override {
+    return "BuildHashOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,
@@ -196,6 +205,10 @@ class BuildHashWorkOrder : public WorkOrder {
 
   ~BuildHashWorkOrder() override {}
 
+  const CatalogRelationSchema& input_relation() const {
+    return input_relation_;
+  }
+
   void execute() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 18ca656..4e05448 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -69,6 +69,10 @@ class CreateIndexOperator : public RelationalOperator {
 
   ~CreateIndexOperator() override {}
 
+  std::string getName() const override {
+    return "CreateIndexOperator";
+  }
+
   /**
    * @note No WorkOrder generated for this operator.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 6d91142..7786cef 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_CREATE_TABLE_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -66,6 +67,10 @@ class CreateTableOperator : public RelationalOperator {
 
   ~CreateTableOperator() override {}
 
+  std::string getName() const override {
+    return "CreateTableOperator";
+  }
+
   /**
    * @note No WorkOrder generated for this operator.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 74da8c1..6bb2075 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_DELETE_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -81,6 +82,10 @@ class DeleteOperator : public RelationalOperator {
 
   ~DeleteOperator() override {}
 
+  std::string getName() const override {
+    return "DeleteOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 181386f..fc48ef9 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -18,6 +18,8 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_DESTROY_HASH_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_DESTROY_HASH_OPERATOR_HPP_
 
+#include <string>
+
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
@@ -58,6 +60,10 @@ class DestroyHashOperator : public RelationalOperator {
 
   ~DestroyHashOperator() override {}
 
+  std::string getName() const override {
+    return "DestroyHashOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index 6c7fca3..ab3344d 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_DROP_TABLE_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -74,6 +75,10 @@ class DropTableOperator : public RelationalOperator {
 
   ~DropTableOperator() override {}
 
+  std::string getName() const override {
+    return "DropTableOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 158a637..af11bc3 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_FINALIZE_AGGREGATION_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -74,6 +75,10 @@ class FinalizeAggregationOperator : public RelationalOperator {
 
   ~FinalizeAggregationOperator() override {}
 
+  std::string getName() const override {
+    return "FinalizeAggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 5d3d7da..235bfe4 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -157,6 +158,29 @@ class HashJoinOperator : public RelationalOperator {
 
   ~HashJoinOperator() override {}
 
+  std::string getName() const override {
+    switch (join_type_) {
+      case JoinType::kInnerJoin:
+        return "HashJoinOperator";
+      case JoinType::kLeftSemiJoin:
+        return "HashJoinOperator(LeftSemi)";
+      case JoinType::kLeftAntiJoin:
+        return "HashJoinOperator(LeftAnti)";
+      case JoinType::kLeftOuterJoin:
+        return "HashJoinOperator(LeftOuter)";
+      default: break;
+    }
+    LOG(FATAL) << "Unknown join type in HashJoinOperator::getName()";
+  }
+
+  const CatalogRelation& build_relation() const {
+    return build_relation_;
+  }
+
+  const CatalogRelation& probe_relation() const {
+    return probe_relation_;
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 78f5199..bf9c56a 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_INSERT_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -73,6 +74,10 @@ class InsertOperator : public RelationalOperator {
 
   ~InsertOperator() override {}
 
+  std::string getName() const override {
+    return "InsertOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 992e76d..041b8e9 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -20,6 +20,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -116,6 +117,10 @@ class NestedLoopsJoinOperator : public RelationalOperator {
 
   ~NestedLoopsJoinOperator() override {}
 
+  std::string getName() const override {
+    return "NestedLoopsJoinOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 116727b..b8d1bd0 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_RELATIONAL_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -55,6 +56,13 @@ class RelationalOperator {
   virtual ~RelationalOperator() {}
 
   /**
+   * @brief Get the name of this relational operator.
+   *
+   * @return The name of this relational operator.
+   */
+  virtual std::string getName() const = 0;
+
+  /**
    * @brief Generate all the next WorkOrders for this RelationalOperator.
    *
    * @note If a RelationalOperator has blocking dependencies, it should not
@@ -226,6 +234,15 @@ class RelationalOperator {
     op_index_ = operator_index;
   }
 
+  /**
+   * @brief Get the index of this operator in the query plan DAG.
+   *
+   * @return The index of this operator in the query plan DAG.
+   */
+  std::size_t getOperatorIndex() const {
+    return op_index_;
+  }
+
  protected:
   /**
    * @brief Constructor

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index f8fe5f6..400a83f 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -20,6 +20,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -93,6 +94,10 @@ class SampleOperator : public RelationalOperator {
 
   ~SampleOperator() override {}
 
+  std::string getName() const override {
+    return "SampleOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 50032b6..d56ee2c 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_SAVE_BLOCKS_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -64,6 +65,10 @@ class SaveBlocksOperator : public RelationalOperator {
 
   ~SaveBlocksOperator() override {}
 
+  std::string getName() const override {
+    return "SaveBlocksOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 0c10686..764dfa3 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_SELECT_OPERATOR_HPP_
 
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -189,6 +190,14 @@ class SelectOperator : public RelationalOperator {
 
   ~SelectOperator() override {}
 
+  std::string getName() const override {
+    return "SelectOperator";
+  }
+
+  const CatalogRelation& input_relation() const {
+    return input_relation_;
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index 177836f..531e269 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -19,6 +19,7 @@
 #define QUICKSTEP_RELATIONAL_OPERATORS_SORT_MERGE_RUN_OPERATOR_HPP_
 
 #include <cstddef>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -129,6 +130,10 @@ class SortMergeRunOperator : public RelationalOperator {
    **/
   ~SortMergeRunOperator() {}
 
+  std::string getName() const override {
+    return "SortMergeRunOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 96a3ce1..d43b90b 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_SORT_RUN_GENERATION_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_SORT_RUN_GENERATION_OPERATOR_HPP_
 
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -109,6 +110,10 @@ class SortRunGenerationOperator : public RelationalOperator {
 
   ~SortRunGenerationOperator() {}
 
+  std::string getName() const override {
+    return "SortRunGenerationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 1b791a6..ad3a9ff 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -19,6 +19,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_TABLE_GENERATOR_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_TABLE_GENERATOR_OPERATOR_HPP_
 
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -76,6 +77,10 @@ class TableGeneratorOperator : public RelationalOperator {
 
   ~TableGeneratorOperator() override {}
 
+  std::string getName() const override {
+    return "TableGeneratorOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 1a62ded..6890d7d 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -134,6 +134,10 @@ class TextScanOperator : public RelationalOperator {
 
   ~TextScanOperator() override {}
 
+  std::string getName() const override {
+    return "TextScanOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index 4471a17..a443b5d 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -20,6 +20,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <string>
 #include <unordered_map>
 #include <vector>
 
@@ -94,6 +95,10 @@ class UpdateOperator : public RelationalOperator {
 
   ~UpdateOperator() override {}
 
+  std::string getName() const override {
+    return "UpdateOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/relational_operators/WindowAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WindowAggregationOperator.hpp b/relational_operators/WindowAggregationOperator.hpp
index bd83248..05632cc 100644
--- a/relational_operators/WindowAggregationOperator.hpp
+++ b/relational_operators/WindowAggregationOperator.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_WINDOW_AGGREGATION_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_WINDOW_AGGREGATION_OPERATOR_HPP_
 
+#include <string>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -78,6 +79,10 @@ class WindowAggregationOperator : public RelationalOperator {
 
   ~WindowAggregationOperator() override {}
 
+  std::string getName() const override {
+    return "WindowAggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 2d3db8f..803b909 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -167,6 +167,9 @@ add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
+add_library(quickstep_utility_ExecutionDAGVisualizer
+            ExecutionDAGVisualizer.cpp
+            ExecutionDAGVisualizer.hpp)
 add_library(quickstep_utility_Glob Glob.cpp Glob.hpp)
 add_library(quickstep_utility_HashPair ../empty_src.cpp HashPair.hpp)
 add_library(quickstep_utility_Macros ../empty_src.cpp Macros.hpp)
@@ -225,6 +228,16 @@ target_link_libraries(quickstep_utility_CheckSnprintf
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_ExecutionDAGVisualizer
+                      quickstep_catalog_CatalogRelationSchema
+                      quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryoptimizer_QueryPlan
+                      quickstep_relationaloperators_AggregationOperator
+                      quickstep_relationaloperators_BuildHashOperator
+                      quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_SelectOperator
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
 target_link_libraries(quickstep_utility_Glob
                       glog)
 target_link_libraries(quickstep_utility_MemStream
@@ -303,6 +316,7 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_DAG
                       quickstep_utility_EqualsAnyConstant
+                      quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Glob
                       quickstep_utility_HashPair
                       quickstep_utility_Macros

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/utility/ExecutionDAGVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/ExecutionDAGVisualizer.cpp b/utility/ExecutionDAGVisualizer.cpp
new file mode 100644
index 0000000..0c0bbb1
--- /dev/null
+++ b/utility/ExecutionDAGVisualizer.cpp
@@ -0,0 +1,230 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#include "utility/ExecutionDAGVisualizer.hpp"
+
+#include <algorithm>
+#include <cmath>
+#include <cstddef>
+#include <iomanip>
+#include <limits>
+#include <set>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogRelationSchema.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_optimizer/QueryPlan.hpp"
+#include "relational_operators/AggregationOperator.hpp"
+#include "relational_operators/BuildHashOperator.hpp"
+#include "relational_operators/HashJoinOperator.hpp"
+#include "relational_operators/SelectOperator.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+ExecutionDAGVisualizer::ExecutionDAGVisualizer(const QueryPlan &plan) {
+  // Do not display these relational operators in the graph.
+  std::set<std::string> no_display_op_names =
+      { "DestroyHashOperator", "DropTableOperator" };
+
+  const auto &dag = plan.getQueryPlanDAG();
+  num_nodes_ = dag.size();
+
+  // Collect DAG vertices info.
+  std::vector<bool> display_ops(num_nodes_, false);
+  for (std::size_t node_index = 0; node_index < num_nodes_; ++node_index) {
+    const auto &node = dag.getNodePayload(node_index);
+    const std::string relop_name = node.getName();
+    if (no_display_op_names.find(relop_name) == no_display_op_names.end()) {
+      display_ops[node_index] = true;
+      NodeInfo &node_info = nodes_[node_index];
+      node_info.id = node_index;
+      node_info.labels.emplace_back(
+          "[" + std::to_string(node.getOperatorIndex()) + "] " + relop_name);
+
+      std::vector<std::pair<std::string, const CatalogRelationSchema*>> input_relations;
+      if (relop_name == "AggregationOperator") {
+        const AggregationOperator &aggregation_op =
+            static_cast<const AggregationOperator&>(node);
+        input_relations.emplace_back("input", &aggregation_op.input_relation());
+      } else if (relop_name == "BuildHashOperator") {
+        const BuildHashOperator &build_hash_op =
+            static_cast<const BuildHashOperator&>(node);
+        input_relations.emplace_back("input", &build_hash_op.input_relation());
+      } else if (relop_name == "HashJoinOperator") {
+        const HashJoinOperator &hash_join_op =
+            static_cast<const HashJoinOperator&>(node);
+        input_relations.emplace_back("probe side", &hash_join_op.probe_relation());
+      } else if (relop_name == "SelectOperator") {
+        const SelectOperator &select_op =
+            static_cast<const SelectOperator&>(node);
+        input_relations.emplace_back("input", &select_op.input_relation());
+      }
+      for (const auto &rel_pair : input_relations) {
+        if (!rel_pair.second->isTemporary()) {
+          node_info.labels.emplace_back(
+              rel_pair.first + " stored relation [" +
+              rel_pair.second->getName() + "]");
+        }
+      }
+    }
+  }
+
+  // Collect DAG edges info.
+  for (std::size_t node_index = 0; node_index < num_nodes_; ++node_index) {
+    if (display_ops[node_index]) {
+      for (const auto &link : dag.getDependents(node_index)) {
+        if (display_ops[link.first]) {
+          edges_.emplace_back();
+          edges_.back().src_node_id = node_index;
+          edges_.back().dst_node_id = link.first;
+          edges_.back().is_pipeline_breaker = link.second;
+        }
+      }
+    }
+  }
+}
+
+void ExecutionDAGVisualizer::bindProfilingStats(
+  const std::vector<WorkOrderTimeEntry> &execution_time_records) {
+  std::vector<std::size_t> time_start(num_nodes_, std::numeric_limits<std::size_t>::max());
+  std::vector<std::size_t> time_end(num_nodes_, 0);
+  std::vector<std::size_t> time_elapsed(num_nodes_, 0);
+  std::size_t overall_start_time = std::numeric_limits<std::size_t>::max();
+  std::size_t overall_end_time = 0;
+  for (const auto &entry : execution_time_records) {
+    const std::size_t relop_index = entry.operator_id;
+    DCHECK_LT(relop_index, num_nodes_);
+
+    const std::size_t workorder_start_time = entry.start_time;
+    const std::size_t workorder_end_time = entry.end_time;
+    overall_start_time = std::min(overall_start_time, workorder_start_time);
+    overall_end_time = std::max(overall_end_time, workorder_end_time);
+
+    time_start[relop_index] =
+        std::min(time_start[relop_index], workorder_start_time);
+    time_end[relop_index] =
+        std::max(time_end[relop_index], workorder_end_time);
+    time_elapsed[relop_index] += (workorder_end_time - workorder_start_time);
+  }
+
+  double total_time_elapsed = 0;
+  for (std::size_t i = 0; i < time_elapsed.size(); ++i) {
+    total_time_elapsed += time_elapsed[i];
+  }
+  std::vector<double> time_percentage(num_nodes_, 0);
+  std::vector<double> span_percentage(num_nodes_, 0);
+  double overall_span = overall_end_time - overall_start_time;
+  double max_percentage = 0;
+  for (std::size_t i = 0; i < time_elapsed.size(); ++i) {
+    time_percentage[i] = time_elapsed[i] / total_time_elapsed * 100;
+    span_percentage[i] = (time_end[i] - time_start[i]) / overall_span * 100;
+    max_percentage = std::max(max_percentage, time_percentage[i] + span_percentage[i]);
+  }
+
+  for (std::size_t node_index = 0; node_index < num_nodes_; ++node_index) {
+    if (nodes_.find(node_index) != nodes_.end()) {
+      const std::size_t relop_start_time = time_start[node_index];
+      const std::size_t relop_end_time = time_end[node_index];
+      const std::size_t relop_elapsed_time = time_elapsed[node_index];
+      NodeInfo &node_info = nodes_[node_index];
+
+      const double hue =
+          (time_percentage[node_index] + span_percentage[node_index]) / max_percentage;
+      node_info.color = std::to_string(hue) + " " + std::to_string(hue) + " 1.0";
+
+      if (overall_start_time == 0) {
+        node_info.labels.emplace_back(
+            "span: " +
+            std::to_string((relop_end_time - relop_start_time) / 1000) + "ms");
+      } else {
+        node_info.labels.emplace_back(
+            "span: [" +
+            std::to_string((relop_start_time - overall_start_time) / 1000) + "ms, " +
+            std::to_string((relop_end_time - overall_start_time) / 1000) + "ms] (" +
+            FormatDigits(span_percentage[node_index], 2) + "%)");
+      }
+
+      node_info.labels.emplace_back(
+          "total: " +
+          std::to_string(relop_elapsed_time / 1000) + "ms (" +
+          FormatDigits(time_percentage[node_index], 2) + "%)");
+
+      const double concurrency =
+          static_cast<double>(relop_elapsed_time) / (relop_end_time - relop_start_time);
+      node_info.labels.emplace_back(
+          "effective concurrency: " + FormatDigits(concurrency, 2));
+    }
+  }
+}
+
+std::string ExecutionDAGVisualizer::toDOT() {
+  // Format output graph
+  std::ostringstream graph_oss;
+  graph_oss << "digraph g {\n";
+  graph_oss << "  rankdir=BT\n";
+  graph_oss << "  node [penwidth=2]\n";
+  graph_oss << "  edge [fontsize=16 fontcolor=gray penwidth=2]\n\n";
+
+  // Format nodes
+  for (const auto &node_pair : nodes_) {
+    const NodeInfo &node_info = node_pair.second;
+    graph_oss << "  " << node_info.id << " [ ";
+    if (!node_info.labels.empty()) {
+      graph_oss << "label=\""
+                << EscapeSpecialChars(JoinToString(node_info.labels, "&#10;"))
+                << "\" ";
+    }
+    if (!node_info.color.empty()) {
+      graph_oss << "style=filled fillcolor=\"" << node_info.color << "\" ";
+    }
+    graph_oss << "]\n";
+  }
+  graph_oss << "\n";
+
+  // Format edges
+  for (const EdgeInfo &edge_info : edges_) {
+    graph_oss << "  " << edge_info.src_node_id << " -> "
+              << edge_info.dst_node_id << " [ ";
+    if (edge_info.is_pipeline_breaker) {
+      graph_oss << "style=dashed ";
+    }
+    if (!edge_info.labels.empty()) {
+      graph_oss << "label=\""
+                << EscapeSpecialChars(JoinToString(edge_info.labels, "&#10;"))
+                << "\" ";
+    }
+    graph_oss << "]\n";
+  }
+  graph_oss << "}\n";
+
+  return graph_oss.str();
+}
+
+std::string ExecutionDAGVisualizer::FormatDigits(const double value,
+                                                 const int num_digits) {
+  std::ostringstream oss;
+  oss << std::fixed << std::setprecision(num_digits) << value;
+  return oss.str();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1b07eaae/utility/ExecutionDAGVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/ExecutionDAGVisualizer.hpp b/utility/ExecutionDAGVisualizer.hpp
new file mode 100644
index 0000000..5c9e434
--- /dev/null
+++ b/utility/ExecutionDAGVisualizer.hpp
@@ -0,0 +1,112 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_EXECUTION_DAG_VISUALIZER_HPP_
+#define QUICKSTEP_UTILITY_EXECUTION_DAG_VISUALIZER_HPP_
+
+#include <cstddef>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class QueryPlan;
+struct WorkOrderTimeEntry;
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A visualizer that converts an execution plan DAG into a graph in
+ *        DOT format. Note that DOT is a plain text graph description language.
+ *
+ * @note This utility tool can be further extended to be more generic.
+ */
+class ExecutionDAGVisualizer {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param plan The execution plan to be visualized.
+   */
+  explicit ExecutionDAGVisualizer(const QueryPlan &plan);
+
+  /**
+   * @brief Destructor
+   */
+  ~ExecutionDAGVisualizer() {}
+
+  /**
+   * @brief Summarize the execution timing stats and bind the stats to the
+   *        corresponding relational operators in the execution plan.
+   *
+   * @param execution_time_records The profiled timing records of execution.
+   */
+  void bindProfilingStats(
+      const std::vector<WorkOrderTimeEntry> &execution_time_records);
+
+  /**
+   * @brief Get the string represenation of the visualized execution plan
+   *        in DOT format (DOT is a plain text graph description language).
+   *
+   * @return The execution plan graph in DOT format.
+   */
+  std::string toDOT();
+
+ private:
+  /**
+   * @brief Format a float value to string representation with the specified
+   *        number of decimal digits.
+   */
+  static std::string FormatDigits(const double value,
+                                  const int num_digits);
+
+  /**
+   * @brief Information of a graph node.
+   */
+  struct NodeInfo {
+    std::size_t id;
+    std::vector<std::string> labels;
+    std::string color;
+  };
+
+  /**
+   * @brief Information of a graph edge.
+   */
+  struct EdgeInfo {
+    std::size_t src_node_id;
+    std::size_t dst_node_id;
+    std::vector<std::string> labels;
+    bool is_pipeline_breaker;
+  };
+
+  std::size_t num_nodes_;
+  std::map<std::size_t, NodeInfo> nodes_;
+  std::vector<EdgeInfo> edges_;
+
+  DISALLOW_COPY_AND_ASSIGN(ExecutionDAGVisualizer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_UTILITY_EXECUTION_DAG_VISUALIZER_HPP_ */


[34/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/TrivialPredicates.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/TrivialPredicates.hpp b/expressions/predicate/TrivialPredicates.hpp
index 273c0a4..4e44976 100644
--- a/expressions/predicate/TrivialPredicates.hpp
+++ b/expressions/predicate/TrivialPredicates.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_PREDICATE_TRIVIAL_PREDICATES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/predicate/tests/Predicate_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/tests/Predicate_unittest.cpp b/expressions/predicate/tests/Predicate_unittest.cpp
index 54c4bb2..3c2889d 100644
--- a/expressions/predicate/tests/Predicate_unittest.cpp
+++ b/expressions/predicate/tests/Predicate_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/scalar/CMakeLists.txt b/expressions/scalar/CMakeLists.txt
index 3c8ace1..8f509da 100644
--- a/expressions/scalar/CMakeLists.txt
+++ b/expressions/scalar/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_expressions_scalar_Scalar Scalar.cpp Scalar.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/Scalar.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/Scalar.cpp b/expressions/scalar/Scalar.cpp
index 8498b9f..a1c436c 100644
--- a/expressions/scalar/Scalar.cpp
+++ b/expressions/scalar/Scalar.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/scalar/Scalar.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/Scalar.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/Scalar.hpp b/expressions/scalar/Scalar.hpp
index 6894fcd..2db850a 100644
--- a/expressions/scalar/Scalar.hpp
+++ b/expressions/scalar/Scalar.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarAttribute.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarAttribute.cpp b/expressions/scalar/ScalarAttribute.cpp
index ed91bd1..08dc9dd 100644
--- a/expressions/scalar/ScalarAttribute.cpp
+++ b/expressions/scalar/ScalarAttribute.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/scalar/ScalarAttribute.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarAttribute.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarAttribute.hpp b/expressions/scalar/ScalarAttribute.hpp
index eda5e8e..c6a41df 100644
--- a/expressions/scalar/ScalarAttribute.hpp
+++ b/expressions/scalar/ScalarAttribute.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_ATTRIBUTE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarBinaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarBinaryExpression.cpp b/expressions/scalar/ScalarBinaryExpression.cpp
index a00fa67..5fe6cde 100644
--- a/expressions/scalar/ScalarBinaryExpression.cpp
+++ b/expressions/scalar/ScalarBinaryExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/scalar/ScalarBinaryExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarBinaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarBinaryExpression.hpp b/expressions/scalar/ScalarBinaryExpression.hpp
index 77de0a3..c84792a 100644
--- a/expressions/scalar/ScalarBinaryExpression.hpp
+++ b/expressions/scalar/ScalarBinaryExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_BINARY_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarCaseExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarCaseExpression.cpp b/expressions/scalar/ScalarCaseExpression.cpp
index 46aa8cd..c81f723 100644
--- a/expressions/scalar/ScalarCaseExpression.cpp
+++ b/expressions/scalar/ScalarCaseExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/scalar/ScalarCaseExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarCaseExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarCaseExpression.hpp b/expressions/scalar/ScalarCaseExpression.hpp
index fc1862b..e6809fa 100644
--- a/expressions/scalar/ScalarCaseExpression.hpp
+++ b/expressions/scalar/ScalarCaseExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_CASE_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarLiteral.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarLiteral.cpp b/expressions/scalar/ScalarLiteral.cpp
index be3acc9..48b5574 100644
--- a/expressions/scalar/ScalarLiteral.cpp
+++ b/expressions/scalar/ScalarLiteral.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/scalar/ScalarLiteral.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarLiteral.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarLiteral.hpp b/expressions/scalar/ScalarLiteral.hpp
index 3ab6671..c7f5ceb 100644
--- a/expressions/scalar/ScalarLiteral.hpp
+++ b/expressions/scalar/ScalarLiteral.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_LITERAL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarUnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.cpp b/expressions/scalar/ScalarUnaryExpression.cpp
index 69d5b49..72fdbe1 100644
--- a/expressions/scalar/ScalarUnaryExpression.cpp
+++ b/expressions/scalar/ScalarUnaryExpression.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/scalar/ScalarUnaryExpression.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/ScalarUnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.hpp b/expressions/scalar/ScalarUnaryExpression.hpp
index 90b360e..608a842 100644
--- a/expressions/scalar/ScalarUnaryExpression.hpp
+++ b/expressions/scalar/ScalarUnaryExpression.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_UNARY_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp b/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
index 399478e..2d1064b 100644
--- a/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
+++ b/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/scalar/tests/Scalar_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/tests/Scalar_unittest.cpp b/expressions/scalar/tests/Scalar_unittest.cpp
index 6cbb672..8e106bb 100644
--- a/expressions/scalar/tests/Scalar_unittest.cpp
+++ b/expressions/scalar/tests/Scalar_unittest.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/table_generator/CMakeLists.txt b/expressions/table_generator/CMakeLists.txt
index e847702..037944e 100644
--- a/expressions/table_generator/CMakeLists.txt
+++ b/expressions/table_generator/CMakeLists.txt
@@ -1,16 +1,19 @@
-#   Copyright 2016 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 QS_PROTOBUF_GENERATE_CPP(expressions_tablegenerator_GeneratorFunction_proto_srcs
                          expressions_tablegenerator_GeneratorFunction_proto_hdrs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GenerateSeries.hpp
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GenerateSeries.hpp b/expressions/table_generator/GenerateSeries.hpp
index 31b3bc9..d749810 100644
--- a/expressions/table_generator/GenerateSeries.hpp
+++ b/expressions/table_generator/GenerateSeries.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_TABLE_GENERATOR_GENERATE_SERIES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GenerateSeriesHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GenerateSeriesHandle.hpp b/expressions/table_generator/GenerateSeriesHandle.hpp
index 093c416..594f12b 100644
--- a/expressions/table_generator/GenerateSeriesHandle.hpp
+++ b/expressions/table_generator/GenerateSeriesHandle.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_TABLE_GENERATOR_GENERATE_SERIES_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GeneratorFunction.hpp
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GeneratorFunction.hpp b/expressions/table_generator/GeneratorFunction.hpp
index 5f3c268..b83a534 100644
--- a/expressions/table_generator/GeneratorFunction.hpp
+++ b/expressions/table_generator/GeneratorFunction.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_TABLE_GENERATOR_GENERATOR_FUNCTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GeneratorFunction.proto
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GeneratorFunction.proto b/expressions/table_generator/GeneratorFunction.proto
index ddb3098..fc50a2f 100644
--- a/expressions/table_generator/GeneratorFunction.proto
+++ b/expressions/table_generator/GeneratorFunction.proto
@@ -1,16 +1,19 @@
-//   Copyright 2016 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GeneratorFunctionFactory.cpp
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GeneratorFunctionFactory.cpp b/expressions/table_generator/GeneratorFunctionFactory.cpp
index e368cb4..b7dec8b 100644
--- a/expressions/table_generator/GeneratorFunctionFactory.cpp
+++ b/expressions/table_generator/GeneratorFunctionFactory.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "expressions/table_generator/GeneratorFunctionFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GeneratorFunctionFactory.hpp
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GeneratorFunctionFactory.hpp b/expressions/table_generator/GeneratorFunctionFactory.hpp
index da2caa6..1d9784a 100644
--- a/expressions/table_generator/GeneratorFunctionFactory.hpp
+++ b/expressions/table_generator/GeneratorFunctionFactory.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_TABLE_GENERATOR_GENERATOR_FUNCTION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/table_generator/GeneratorFunctionHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/table_generator/GeneratorFunctionHandle.hpp b/expressions/table_generator/GeneratorFunctionHandle.hpp
index 9a0b3b4..b2fc068 100644
--- a/expressions/table_generator/GeneratorFunctionHandle.hpp
+++ b/expressions/table_generator/GeneratorFunctionHandle.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_EXPRESSIONS_TABLE_GENERATOR_GENERATOR_FUNCTION_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/CMakeLists.txt b/expressions/window_aggregation/CMakeLists.txt
index 3a79b7e..b33a401 100644
--- a/expressions/window_aggregation/CMakeLists.txt
+++ b/expressions/window_aggregation/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   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.
+# 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.
 
 QS_PROTOBUF_GENERATE_CPP(expressions_windowaggregation_WindowAggregateFunction_proto_srcs
                          expressions_windowaggregation_WindowAggregateFunction_proto_hdrs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunction.cpp b/expressions/window_aggregation/WindowAggregateFunction.cpp
index 3911e1c..db208a9 100644
--- a/expressions/window_aggregation/WindowAggregateFunction.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunction.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunction.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunction.hpp b/expressions/window_aggregation/WindowAggregateFunction.hpp
index 7ffc4ae..2b26ccf 100644
--- a/expressions/window_aggregation/WindowAggregateFunction.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunction.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_HPP_
@@ -56,7 +56,7 @@ class WindowAggregationHandle;
  * WindowAggregationHandle object, then use the methods of
  * WindowAggregationHandle to do the actual window aggregation. Finally, delete
  * the WindowAggregationHandle after finished.
- * 
+ *
  * See WindowAggregationHandle for more detailed information about how
  * window aggregates are actually computed.
  **/
@@ -127,7 +127,7 @@ class WindowAggregateFunction {
    * @param is_row True if the frame mode is ROWS, false if RANGE.
    * @param num_preceding The number of rows/range that precedes the current row.
    * @param num_following The number of rows/range that follows the current row.
-   * 
+   *
    * @return A new WindowAggregationHandle that can be used to compute this
    *         WindowAggregateFunction over the specified window definition.
    *         Caller is responsible for deleting the returned object.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunction.proto
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunction.proto b/expressions/window_aggregation/WindowAggregateFunction.proto
index fe8d799..8667422 100644
--- a/expressions/window_aggregation/WindowAggregateFunction.proto
+++ b/expressions/window_aggregation/WindowAggregateFunction.proto
@@ -1,19 +1,19 @@
-//   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.
+// 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.
 
 syntax = "proto2";
 
@@ -25,7 +25,7 @@ message WindowAggregateFunction {
     COUNT = 1;
     MAX = 2;
     MIN = 3;
-    SUM = 4;    
+    SUM = 4;
   }
 
   required WindowAggregationID window_aggregation_id = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp b/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
index beb1c7a..20c296b 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunctionAvg.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp b/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
index 0e50415..1706ce8 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_AVG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionCount.cpp b/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
index ccd81ac..9290ac0 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunctionCount.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionCount.hpp b/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
index 2e5506a..f508105 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_COUNT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp b/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
index 65247f2..bb5d02e 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunctionFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp b/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
index 1d59e93..e572b13 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMax.cpp b/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
index acfce82..594301a 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunctionMax.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMax.hpp b/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
index a215703..15563df 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MAX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMin.cpp b/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
index cd845bd..650241f 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunctionMin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMin.hpp b/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
index fab88a8..8bb4386 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionSum.cpp b/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
index e2aeb60..14c51d8 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregateFunctionSum.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionSum.hpp b/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
index 8d7d61d..89582dd 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_SUM_HPP_


[33/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregationHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandle.cpp b/expressions/window_aggregation/WindowAggregationHandle.cpp
index 835eaff..f26656d 100644
--- a/expressions/window_aggregation/WindowAggregationHandle.cpp
+++ b/expressions/window_aggregation/WindowAggregationHandle.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregationHandle.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandle.hpp b/expressions/window_aggregation/WindowAggregationHandle.hpp
index 41d1d96..3569123 100644
--- a/expressions/window_aggregation/WindowAggregationHandle.hpp
+++ b/expressions/window_aggregation/WindowAggregationHandle.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandleAvg.cpp b/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
index e6a4b3f..b1c6e3b 100644
--- a/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
+++ b/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 "expressions/window_aggregation/WindowAggregationHandleAvg.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandleAvg.hpp b/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
index f7f2e4d..e3885f0 100644
--- a/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
+++ b/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_AVG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/WindowAggregationID.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationID.hpp b/expressions/window_aggregation/WindowAggregationID.hpp
index 8122df3..da90357 100644
--- a/expressions/window_aggregation/WindowAggregationID.hpp
+++ b/expressions/window_aggregation/WindowAggregationID.hpp
@@ -1,20 +1,20 @@
 /**
- *   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.
+ * 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 QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp b/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
index cb58083..6e1364a 100644
--- a/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
+++ b/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
@@ -1,21 +1,20 @@
 /**
- *   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.
- *   limitations under the License.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 32ea1a9..b1bd870 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Although flex option nounistd works fine for the generated cpp file, we need
 # to make sure this is also set for the header.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseAssignment.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseAssignment.hpp b/parser/ParseAssignment.hpp
index 65066ae..eea0a7c 100644
--- a/parser/ParseAssignment.hpp
+++ b/parser/ParseAssignment.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_ASSIGNMENT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseAttributeDefinition.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseAttributeDefinition.cpp b/parser/ParseAttributeDefinition.cpp
index 9772531..ec6508b 100644
--- a/parser/ParseAttributeDefinition.cpp
+++ b/parser/ParseAttributeDefinition.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseAttributeDefinition.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseAttributeDefinition.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseAttributeDefinition.hpp b/parser/ParseAttributeDefinition.hpp
index f83c908..5c20d13 100644
--- a/parser/ParseAttributeDefinition.hpp
+++ b/parser/ParseAttributeDefinition.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_ATTRIBUTE_DEFINITION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseBasicExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index bbb6801..b0b1247 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseBasicExpressions.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseBasicExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index 64ac119..d8de669 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_BASIC_EXPRESSIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseBlockProperties.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBlockProperties.cpp b/parser/ParseBlockProperties.cpp
index dd9280c..31c30a0 100644
--- a/parser/ParseBlockProperties.cpp
+++ b/parser/ParseBlockProperties.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseBlockProperties.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseBlockProperties.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBlockProperties.hpp b/parser/ParseBlockProperties.hpp
index 4b3f4b1..ce0cd92 100644
--- a/parser/ParseBlockProperties.hpp
+++ b/parser/ParseBlockProperties.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_BLOCK_PROPERTIES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseCaseExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseCaseExpressions.cpp b/parser/ParseCaseExpressions.cpp
index 800266d..47f2a33 100644
--- a/parser/ParseCaseExpressions.cpp
+++ b/parser/ParseCaseExpressions.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseCaseExpressions.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseCaseExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseCaseExpressions.hpp b/parser/ParseCaseExpressions.hpp
index 91d815e..6bc0b00 100644
--- a/parser/ParseCaseExpressions.hpp
+++ b/parser/ParseCaseExpressions.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_CASE_EXPRESSIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseExpression.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseExpression.hpp b/parser/ParseExpression.hpp
index 3541f83..1b9ade4 100644
--- a/parser/ParseExpression.hpp
+++ b/parser/ParseExpression.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseGeneratorTableReference.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseGeneratorTableReference.cpp b/parser/ParseGeneratorTableReference.cpp
index 6ae686e..d8f5a48 100644
--- a/parser/ParseGeneratorTableReference.cpp
+++ b/parser/ParseGeneratorTableReference.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseGeneratorTableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseGeneratorTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseGeneratorTableReference.hpp b/parser/ParseGeneratorTableReference.hpp
index 82de5e1..4604b87 100644
--- a/parser/ParseGeneratorTableReference.hpp
+++ b/parser/ParseGeneratorTableReference.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_GENERATOR_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseGroupBy.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseGroupBy.cpp b/parser/ParseGroupBy.cpp
index b7bb64a..324a4c6 100644
--- a/parser/ParseGroupBy.cpp
+++ b/parser/ParseGroupBy.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseGroupBy.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseGroupBy.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseGroupBy.hpp b/parser/ParseGroupBy.hpp
index caa5d8f..1e36cd0 100644
--- a/parser/ParseGroupBy.hpp
+++ b/parser/ParseGroupBy.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_GROUP_BY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseHaving.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseHaving.cpp b/parser/ParseHaving.cpp
index dc817fc..277f922 100644
--- a/parser/ParseHaving.cpp
+++ b/parser/ParseHaving.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseHaving.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseHaving.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseHaving.hpp b/parser/ParseHaving.hpp
index 95336a3..1059a7a 100644
--- a/parser/ParseHaving.hpp
+++ b/parser/ParseHaving.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_HAVING_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseIndexProperties.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseIndexProperties.cpp b/parser/ParseIndexProperties.cpp
index fe88513..8268b5c 100644
--- a/parser/ParseIndexProperties.cpp
+++ b/parser/ParseIndexProperties.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseIndexProperties.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseIndexProperties.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseIndexProperties.hpp b/parser/ParseIndexProperties.hpp
index 2cb5df9..1ff13be 100644
--- a/parser/ParseIndexProperties.hpp
+++ b/parser/ParseIndexProperties.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_INDEX_PROPERTIES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseJoinedTableReference.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseJoinedTableReference.cpp b/parser/ParseJoinedTableReference.cpp
index c1bc956..b2cbeb2 100644
--- a/parser/ParseJoinedTableReference.cpp
+++ b/parser/ParseJoinedTableReference.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseJoinedTableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseJoinedTableReference.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseJoinedTableReference.hpp b/parser/ParseJoinedTableReference.hpp
index 182f472..f1f4aa2 100644
--- a/parser/ParseJoinedTableReference.hpp
+++ b/parser/ParseJoinedTableReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_JOINED_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseKeyValue.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseKeyValue.hpp b/parser/ParseKeyValue.hpp
index 0a28cab..9d6d511 100644
--- a/parser/ParseKeyValue.hpp
+++ b/parser/ParseKeyValue.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_KEY_VALUE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseLimit.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseLimit.cpp b/parser/ParseLimit.cpp
index e83ccb8..f39c430 100644
--- a/parser/ParseLimit.cpp
+++ b/parser/ParseLimit.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseLimit.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseLimit.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseLimit.hpp b/parser/ParseLimit.hpp
index 40d34aa..2afb049 100644
--- a/parser/ParseLimit.hpp
+++ b/parser/ParseLimit.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_LIMIT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseLiteralValue.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseLiteralValue.cpp b/parser/ParseLiteralValue.cpp
index f839bed..d4753ab 100644
--- a/parser/ParseLiteralValue.cpp
+++ b/parser/ParseLiteralValue.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseLiteralValue.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseLiteralValue.hpp b/parser/ParseLiteralValue.hpp
index ff6abac..105a774 100644
--- a/parser/ParseLiteralValue.hpp
+++ b/parser/ParseLiteralValue.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_LITERAL_VALUE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseOrderBy.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseOrderBy.cpp b/parser/ParseOrderBy.cpp
index 314aa50..da46a1f 100644
--- a/parser/ParseOrderBy.cpp
+++ b/parser/ParseOrderBy.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseOrderBy.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseOrderBy.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseOrderBy.hpp b/parser/ParseOrderBy.hpp
index 88a2856..88e9679 100644
--- a/parser/ParseOrderBy.hpp
+++ b/parser/ParseOrderBy.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_ORDERBY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParsePartitionClause.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePartitionClause.hpp b/parser/ParsePartitionClause.hpp
index 1b7096f..d2912e7 100644
--- a/parser/ParsePartitionClause.hpp
+++ b/parser/ParsePartitionClause.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_PARTITION_CLAUSE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParsePredicate.cpp
----------------------------------------------------------------------
diff --git a/parser/ParsePredicate.cpp b/parser/ParsePredicate.cpp
index ad786b5..9747b08 100644
--- a/parser/ParsePredicate.cpp
+++ b/parser/ParsePredicate.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParsePredicate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParsePredicate.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePredicate.hpp b/parser/ParsePredicate.hpp
index 2d65142..cc3e66d 100644
--- a/parser/ParsePredicate.hpp
+++ b/parser/ParsePredicate.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParsePredicateExists.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePredicateExists.hpp b/parser/ParsePredicateExists.hpp
index 1e02d2f..1738964 100644
--- a/parser/ParsePredicateExists.hpp
+++ b/parser/ParsePredicateExists.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_PREDICATE_EXISTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParsePredicateInTableQuery.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePredicateInTableQuery.hpp b/parser/ParsePredicateInTableQuery.hpp
index 423ec3a..d669522 100644
--- a/parser/ParsePredicateInTableQuery.hpp
+++ b/parser/ParsePredicateInTableQuery.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_PREDICATE_IN_TABLE_QUERY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParsePriority.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePriority.hpp b/parser/ParsePriority.hpp
index 89806d4..e53fbfa 100644
--- a/parser/ParsePriority.hpp
+++ b/parser/ParsePriority.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_PRIORITY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSample.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSample.cpp b/parser/ParseSample.cpp
index 4619072..47e69e6 100644
--- a/parser/ParseSample.cpp
+++ b/parser/ParseSample.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseSample.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSample.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSample.hpp b/parser/ParseSample.hpp
index 3a70b33..e2b5672 100644
--- a/parser/ParseSample.hpp
+++ b/parser/ParseSample.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_SAMPLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSelect.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSelect.hpp b/parser/ParseSelect.hpp
index 930a215..d6732e6 100644
--- a/parser/ParseSelect.hpp
+++ b/parser/ParseSelect.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_SELECT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSelectionClause.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSelectionClause.cpp b/parser/ParseSelectionClause.cpp
index 048336a..1e9a807 100644
--- a/parser/ParseSelectionClause.cpp
+++ b/parser/ParseSelectionClause.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "parser/ParseSelectionClause.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/parser/ParseSelectionClause.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSelectionClause.hpp b/parser/ParseSelectionClause.hpp
index 91bd6af..f636f36 100644
--- a/parser/ParseSelectionClause.hpp
+++ b/parser/ParseSelectionClause.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_PARSER_PARSE_SELECTION_CLAUSE_HPP_



[13/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/NotEqualComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/NotEqualComparison.hpp b/types/operations/comparisons/NotEqualComparison.hpp
index 995eaf3..560f613 100644
--- a/types/operations/comparisons/NotEqualComparison.hpp
+++ b/types/operations/comparisons/NotEqualComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_NOT_EQUAL_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/PatternMatchingComparators-inl.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/PatternMatchingComparators-inl.hpp b/types/operations/comparisons/PatternMatchingComparators-inl.hpp
index 6188a52..617eadf 100644
--- a/types/operations/comparisons/PatternMatchingComparators-inl.hpp
+++ b/types/operations/comparisons/PatternMatchingComparators-inl.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_PATTERN_MATCHING_COMPARATORS_INL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/PatternMatchingComparators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/PatternMatchingComparators.hpp b/types/operations/comparisons/PatternMatchingComparators.hpp
index ed105a6..bb2af97 100644
--- a/types/operations/comparisons/PatternMatchingComparators.hpp
+++ b/types/operations/comparisons/PatternMatchingComparators.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_PATTERN_MATCHING_COMPARATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/PatternMatchingComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/PatternMatchingComparison.cpp b/types/operations/comparisons/PatternMatchingComparison.cpp
index 052f0f0..4207f0f 100644
--- a/types/operations/comparisons/PatternMatchingComparison.cpp
+++ b/types/operations/comparisons/PatternMatchingComparison.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/comparisons/PatternMatchingComparison.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/PatternMatchingComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/PatternMatchingComparison.hpp b/types/operations/comparisons/PatternMatchingComparison.hpp
index b4162f0..bfd0b8c 100644
--- a/types/operations/comparisons/PatternMatchingComparison.hpp
+++ b/types/operations/comparisons/PatternMatchingComparison.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_COMPARISONS_PATTERN_MATCHING_COMPARISON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/comparisons/tests/Comparison_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/tests/Comparison_unittest.cpp b/types/operations/comparisons/tests/Comparison_unittest.cpp
index d09aab7..82db0f2 100644
--- a/types/operations/comparisons/tests/Comparison_unittest.cpp
+++ b/types/operations/comparisons/tests/Comparison_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <algorithm>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp b/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
index 1e61db3..c10d5cf 100644
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
+++ b/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.hpp b/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
index 2c6009a..5eed073 100644
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
+++ b/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp b/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
index 176b5b3..bf3f7b6 100644
--- a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
+++ b/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMakeLists.txt b/types/operations/unary_operations/CMakeLists.txt
index 5c54d9c..d612464 100644
--- a/types/operations/unary_operations/CMakeLists.txt
+++ b/types/operations/unary_operations/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations ArithmeticUnaryOperations.cpp ArithmeticUnaryOperations.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/DateExtractOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/DateExtractOperation.cpp b/types/operations/unary_operations/DateExtractOperation.cpp
index 525ba00..ad8a0aa 100644
--- a/types/operations/unary_operations/DateExtractOperation.cpp
+++ b/types/operations/unary_operations/DateExtractOperation.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/unary_operations/DateExtractOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/DateExtractOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/DateExtractOperation.hpp b/types/operations/unary_operations/DateExtractOperation.hpp
index dcda575..fd12bed 100644
--- a/types/operations/unary_operations/DateExtractOperation.hpp
+++ b/types/operations/unary_operations/DateExtractOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_DATE_EXTRACT_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/NumericCastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/NumericCastOperation.hpp b/types/operations/unary_operations/NumericCastOperation.hpp
index 6662796..1c5e3d4 100644
--- a/types/operations/unary_operations/NumericCastOperation.hpp
+++ b/types/operations/unary_operations/NumericCastOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/SubstringOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.cpp b/types/operations/unary_operations/SubstringOperation.cpp
index 463cd33..84f1c8d 100644
--- a/types/operations/unary_operations/SubstringOperation.cpp
+++ b/types/operations/unary_operations/SubstringOperation.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/unary_operations/SubstringOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/SubstringOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.hpp b/types/operations/unary_operations/SubstringOperation.hpp
index d215eae..66f311f 100644
--- a/types/operations/unary_operations/SubstringOperation.hpp
+++ b/types/operations/unary_operations/SubstringOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_SUBSTRING_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/UnaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.cpp b/types/operations/unary_operations/UnaryOperation.cpp
index 6721bd6..af150b3 100644
--- a/types/operations/unary_operations/UnaryOperation.cpp
+++ b/types/operations/unary_operations/UnaryOperation.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/unary_operations/UnaryOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/UnaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.hpp b/types/operations/unary_operations/UnaryOperation.hpp
index 9542231..30a2961 100644
--- a/types/operations/unary_operations/UnaryOperation.hpp
+++ b/types/operations/unary_operations/UnaryOperation.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/UnaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.cpp b/types/operations/unary_operations/UnaryOperationFactory.cpp
index 13cb76e..b306061 100644
--- a/types/operations/unary_operations/UnaryOperationFactory.cpp
+++ b/types/operations/unary_operations/UnaryOperationFactory.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/unary_operations/UnaryOperationFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/UnaryOperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.hpp b/types/operations/unary_operations/UnaryOperationFactory.hpp
index 59c439d..2ce83d4 100644
--- a/types/operations/unary_operations/UnaryOperationFactory.hpp
+++ b/types/operations/unary_operations/UnaryOperationFactory.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/UnaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.cpp b/types/operations/unary_operations/UnaryOperationID.cpp
index dae64d3..b47a848 100644
--- a/types/operations/unary_operations/UnaryOperationID.cpp
+++ b/types/operations/unary_operations/UnaryOperationID.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/operations/unary_operations/UnaryOperationID.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/UnaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.hpp b/types/operations/unary_operations/UnaryOperationID.hpp
index 9cbb6e4..fa50f50 100644
--- a/types/operations/unary_operations/UnaryOperationID.hpp
+++ b/types/operations/unary_operations/UnaryOperationID.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/tests/DateExtractOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/tests/DateExtractOperation_unittest.cpp b/types/operations/unary_operations/tests/DateExtractOperation_unittest.cpp
index 745efc6..ec14408 100644
--- a/types/operations/unary_operations/tests/DateExtractOperation_unittest.cpp
+++ b/types/operations/unary_operations/tests/DateExtractOperation_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstdint>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/tests/NegateUnaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/tests/NegateUnaryOperation_unittest.cpp b/types/operations/unary_operations/tests/NegateUnaryOperation_unittest.cpp
index 81e5d7a..b49fd30 100644
--- a/types/operations/unary_operations/tests/NegateUnaryOperation_unittest.cpp
+++ b/types/operations/unary_operations/tests/NegateUnaryOperation_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/Type.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/tests/NumericCastOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/tests/NumericCastOperation_unittest.cpp b/types/operations/unary_operations/tests/NumericCastOperation_unittest.cpp
index ec2c1b5..fcad02f 100644
--- a/types/operations/unary_operations/tests/NumericCastOperation_unittest.cpp
+++ b/types/operations/unary_operations/tests/NumericCastOperation_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/Type.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/operations/unary_operations/tests/UnaryOperation_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/tests/UnaryOperation_unittest.cpp b/types/operations/unary_operations/tests/UnaryOperation_unittest.cpp
index fc69394..0f4dc7a 100644
--- a/types/operations/unary_operations/tests/UnaryOperation_unittest.cpp
+++ b/types/operations/unary_operations/tests/UnaryOperation_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/port/CMakeLists.txt b/types/port/CMakeLists.txt
index 2395924..f013b44 100644
--- a/types/port/CMakeLists.txt
+++ b/types/port/CMakeLists.txt
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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(CheckCXXSymbolExists)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/TypesPortConfig.h.in
----------------------------------------------------------------------
diff --git a/types/port/TypesPortConfig.h.in b/types/port/TypesPortConfig.h.in
index c8a7426..a808a1a 100644
--- a/types/port/TypesPortConfig.h.in
+++ b/types/port/TypesPortConfig.h.in
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #cmakedefine QUICKSTEP_HAVE_GMTIME_R

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/gmtime_r.hpp
----------------------------------------------------------------------
diff --git a/types/port/gmtime_r.hpp b/types/port/gmtime_r.hpp
index a205cc7..3a632aa 100644
--- a/types/port/gmtime_r.hpp
+++ b/types/port/gmtime_r.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_PORT_GMTIME_R_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/localtime_r.hpp
----------------------------------------------------------------------
diff --git a/types/port/localtime_r.hpp b/types/port/localtime_r.hpp
index cdf907d..2c6853c 100644
--- a/types/port/localtime_r.hpp
+++ b/types/port/localtime_r.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_PORT_LOCALTIME_R_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/strnlen.hpp
----------------------------------------------------------------------
diff --git a/types/port/strnlen.hpp b/types/port/strnlen.hpp
index da7985f..5327d2d 100644
--- a/types/port/strnlen.hpp
+++ b/types/port/strnlen.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_PORT_STRNLEN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/tests/timegm_benchmark.cpp
----------------------------------------------------------------------
diff --git a/types/port/tests/timegm_benchmark.cpp b/types/port/tests/timegm_benchmark.cpp
index 005077b..38a64c2 100644
--- a/types/port/tests/timegm_benchmark.cpp
+++ b/types/port/tests/timegm_benchmark.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/tests/timegm_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/port/tests/timegm_unittest.cpp b/types/port/tests/timegm_unittest.cpp
index 88ed49e..3b13dcf 100644
--- a/types/port/tests/timegm_unittest.cpp
+++ b/types/port/tests/timegm_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/timegm.cpp
----------------------------------------------------------------------
diff --git a/types/port/timegm.cpp b/types/port/timegm.cpp
index 8c27fec..4d48788 100644
--- a/types/port/timegm.cpp
+++ b/types/port/timegm.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "types/port/timegm.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/port/timegm.hpp
----------------------------------------------------------------------
diff --git a/types/port/timegm.hpp b/types/port/timegm.hpp
index 0a02e57..dcb1663 100644
--- a/types/port/timegm.hpp
+++ b/types/port/timegm.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, 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.
+ * 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 QUICKSTEP_TYPES_PORT_TIMEGM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/CharType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/CharType_unittest.cpp b/types/tests/CharType_unittest.cpp
index 390d002..029bd5e 100644
--- a/types/tests/CharType_unittest.cpp
+++ b/types/tests/CharType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/DateType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/DateType_unittest.cpp b/types/tests/DateType_unittest.cpp
index 637f369..44168aa 100644
--- a/types/tests/DateType_unittest.cpp
+++ b/types/tests/DateType_unittest.cpp
@@ -1,15 +1,20 @@
 /**
- *   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
+ * 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
+ *   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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/DatetimeIntervalType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/DatetimeIntervalType_unittest.cpp b/types/tests/DatetimeIntervalType_unittest.cpp
index 5156e1d..cb85232 100644
--- a/types/tests/DatetimeIntervalType_unittest.cpp
+++ b/types/tests/DatetimeIntervalType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/DatetimeType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/DatetimeType_unittest.cpp b/types/tests/DatetimeType_unittest.cpp
index 81ce0b5..f5c8dd4 100644
--- a/types/tests/DatetimeType_unittest.cpp
+++ b/types/tests/DatetimeType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/DoubleType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/DoubleType_unittest.cpp b/types/tests/DoubleType_unittest.cpp
index 50b250d..9261de7 100644
--- a/types/tests/DoubleType_unittest.cpp
+++ b/types/tests/DoubleType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/FloatType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/FloatType_unittest.cpp b/types/tests/FloatType_unittest.cpp
index 146cce6..a735941 100644
--- a/types/tests/FloatType_unittest.cpp
+++ b/types/tests/FloatType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/IntType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/IntType_unittest.cpp b/types/tests/IntType_unittest.cpp
index 9b2f1b1..774854f 100644
--- a/types/tests/IntType_unittest.cpp
+++ b/types/tests/IntType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/LongType_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/LongType_unittest.cpp b/types/tests/LongType_unittest.cpp
index 3f9e313..cb20e65 100644
--- a/types/tests/LongType_unittest.cpp
+++ b/types/tests/LongType_unittest.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/TypeTest_common.hpp
----------------------------------------------------------------------
diff --git a/types/tests/TypeTest_common.hpp b/types/tests/TypeTest_common.hpp
index d185405..74ac4f8 100644
--- a/types/tests/TypeTest_common.hpp
+++ b/types/tests/TypeTest_common.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_TYPES_TESTS_TYPE_TEST_COMMON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/types/tests/Type_unittest.cpp
----------------------------------------------------------------------
diff --git a/types/tests/Type_unittest.cpp b/types/tests/Type_unittest.cpp
index c2f502b..a2bd361 100644
--- a/types/tests/Type_unittest.cpp
+++ b/types/tests/Type_unittest.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <cstddef>



[22/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/DataExchangerAsync.hpp
----------------------------------------------------------------------
diff --git a/storage/DataExchangerAsync.hpp b/storage/DataExchangerAsync.hpp
index 75a4e4d..f3b256a 100644
--- a/storage/DataExchangerAsync.hpp
+++ b/storage/DataExchangerAsync.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_DATA_EXCHANGER_ASYNC_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/EvictionPolicy.cpp
----------------------------------------------------------------------
diff --git a/storage/EvictionPolicy.cpp b/storage/EvictionPolicy.cpp
index b7e0033..2f10f09 100644
--- a/storage/EvictionPolicy.cpp
+++ b/storage/EvictionPolicy.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/EvictionPolicy.hpp"
@@ -310,12 +312,12 @@ class LRUKEvictionPolicyImpl : public EvictionPolicy {
 #ifdef QUICKSTEP_DEBUG
   /**
    * @brief Prints the contents of the LRUKEvictionPolicyImpl. Use this
-   *        strictly for debugging purposes. 
+   *        strictly for debugging purposes.
    *        WARNING: This method is not safe as it does not acquire
-   *                 any mutexes on the internal state variable, so use it 
+   *                 any mutexes on the internal state variable, so use it
    *                 primarily for debugging purposes.
    *
-   * @param print_map If true, print the eviction map. 
+   * @param print_map If true, print the eviction map.
    * @param print_nr_list If true, print the list of non referenced blocks.
    *
    */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/EvictionPolicy.hpp
----------------------------------------------------------------------
diff --git a/storage/EvictionPolicy.hpp b/storage/EvictionPolicy.hpp
index ae2a258..9c98fcd 100644
--- a/storage/EvictionPolicy.hpp
+++ b/storage/EvictionPolicy.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_EVICTION_POLICY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManager.hpp
----------------------------------------------------------------------
diff --git a/storage/FileManager.hpp b/storage/FileManager.hpp
index b179071..74a59da 100644
--- a/storage/FileManager.hpp
+++ b/storage/FileManager.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_FILE_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerHdfs.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.cpp b/storage/FileManagerHdfs.cpp
index e8f048b..e28ed3b 100644
--- a/storage/FileManagerHdfs.cpp
+++ b/storage/FileManagerHdfs.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #define __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerHdfs.hpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.hpp b/storage/FileManagerHdfs.hpp
index 440a5e4..3c751b5 100644
--- a/storage/FileManagerHdfs.hpp
+++ b/storage/FileManagerHdfs.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_FILE_MANAGER_HDFS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerLocal.hpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerLocal.hpp b/storage/FileManagerLocal.hpp
index 4f9f7b1..55ed6ab 100644
--- a/storage/FileManagerLocal.hpp
+++ b/storage/FileManagerLocal.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_FILE_MANAGER_LOCAL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerPosix.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerPosix.cpp b/storage/FileManagerPosix.cpp
index 0346f0d..43a5a66 100644
--- a/storage/FileManagerPosix.cpp
+++ b/storage/FileManagerPosix.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #define __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerPosix.hpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerPosix.hpp b/storage/FileManagerPosix.hpp
index b2aea27..a065605 100644
--- a/storage/FileManagerPosix.hpp
+++ b/storage/FileManagerPosix.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_FILE_MANAGER_POSIX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerWindows.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerWindows.cpp b/storage/FileManagerWindows.cpp
index b1763ec..6ca0070 100644
--- a/storage/FileManagerWindows.cpp
+++ b/storage/FileManagerWindows.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 #define __STDC_FORMAT_MACROS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/FileManagerWindows.hpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerWindows.hpp b/storage/FileManagerWindows.hpp
index e7b3aa8..a87227a 100644
--- a/storage/FileManagerWindows.hpp
+++ b/storage/FileManagerWindows.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_FILE_MANAGER_WINDOWS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index be31fd9..9fa41a2 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index 7f00f29..ade30d8 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -1,19 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
-//   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-//    University of Wisconsin\u2014Madison.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index c37c1ec..f1594e3 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_HASH_TABLE_BASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTableFactory.cpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.cpp b/storage/HashTableFactory.cpp
index 5bc19e6..c93ce98 100644
--- a/storage/HashTableFactory.cpp
+++ b/storage/HashTableFactory.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/HashTableFactory.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 34baaeb..40b39de 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_HASH_TABLE_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTableKeyManager.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableKeyManager.hpp b/storage/HashTableKeyManager.hpp
index 98485dc..0295ad4 100644
--- a/storage/HashTableKeyManager.hpp
+++ b/storage/HashTableKeyManager.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_HASH_TABLE_KEY_MANAGER_HPP_
@@ -199,7 +202,7 @@ class HashTableKeyManager {
    * @param bucket A pointer to the bucket in the HashTable to write the key
    *        to.
    * @param prealloc_state If non-null, variable length key components will be
-   *        located in preallocated variable-length storage. 
+   *        located in preallocated variable-length storage.
    **/
   inline void writeKeyComponentToBucket(
       const TypedValue &component,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index c16d0f1..53fe514 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/IndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/IndexSubBlock.hpp b/storage/IndexSubBlock.hpp
index 5a1cfff..26de568 100644
--- a/storage/IndexSubBlock.hpp
+++ b/storage/IndexSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_INDEX_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/IndexSubBlockDescriptionFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/IndexSubBlockDescriptionFactory.hpp b/storage/IndexSubBlockDescriptionFactory.hpp
index 5e7f5a1..1513c74 100644
--- a/storage/IndexSubBlockDescriptionFactory.hpp
+++ b/storage/IndexSubBlockDescriptionFactory.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_INDEX_SUB_BLOCK_DESCRIPTION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 2866c5f..9897aed 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/InsertDestination.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 5ff33f5..3dae9a0 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_INSERT_DESTINATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/InsertDestination.proto
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.proto b/storage/InsertDestination.proto
index 6083539..aaffa2e 100644
--- a/storage/InsertDestination.proto
+++ b/storage/InsertDestination.proto
@@ -1,17 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/InsertDestinationInterface.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestinationInterface.hpp b/storage/InsertDestinationInterface.hpp
index a66a149..423dff1 100644
--- a/storage/InsertDestinationInterface.hpp
+++ b/storage/InsertDestinationInterface.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_INSERT_DESTINATION_INTERFACE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/LinearOpenAddressingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/LinearOpenAddressingHashTable.hpp b/storage/LinearOpenAddressingHashTable.hpp
index 438a2d1..4953b4d 100644
--- a/storage/LinearOpenAddressingHashTable.hpp
+++ b/storage/LinearOpenAddressingHashTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_LINEAR_OPEN_ADDRESSING_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/PackedRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.cpp b/storage/PackedRowStoreTupleStorageSubBlock.cpp
index ef83a29..0ad4a4c 100644
--- a/storage/PackedRowStoreTupleStorageSubBlock.cpp
+++ b/storage/PackedRowStoreTupleStorageSubBlock.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/PackedRowStoreTupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/PackedRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.hpp b/storage/PackedRowStoreTupleStorageSubBlock.hpp
index 9a208dc..0cd41f4 100644
--- a/storage/PackedRowStoreTupleStorageSubBlock.hpp
+++ b/storage/PackedRowStoreTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
index 03a975e..80edecd 100644
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ b/storage/PackedRowStoreValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/PreloaderThread.cpp
----------------------------------------------------------------------
diff --git a/storage/PreloaderThread.cpp b/storage/PreloaderThread.cpp
index 4bfa9c5..7f97a05 100644
--- a/storage/PreloaderThread.cpp
+++ b/storage/PreloaderThread.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/PreloaderThread.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/PreloaderThread.hpp
----------------------------------------------------------------------
diff --git a/storage/PreloaderThread.hpp b/storage/PreloaderThread.hpp
index 16df34a..aaea790 100644
--- a/storage/PreloaderThread.hpp
+++ b/storage/PreloaderThread.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_PRELOADER_THREAD_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SMAIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SMAIndexSubBlock.cpp b/storage/SMAIndexSubBlock.cpp
index 3a1ccc2..621bafd 100644
--- a/storage/SMAIndexSubBlock.cpp
+++ b/storage/SMAIndexSubBlock.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/SMAIndexSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SMAIndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/SMAIndexSubBlock.hpp b/storage/SMAIndexSubBlock.hpp
index 895add9..c8fb3b3 100644
--- a/storage/SMAIndexSubBlock.hpp
+++ b/storage/SMAIndexSubBlock.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SMA_INDEX_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/SeparateChainingHashTable.hpp b/storage/SeparateChainingHashTable.hpp
index c096b1b..3ca060d 100644
--- a/storage/SeparateChainingHashTable.hpp
+++ b/storage/SeparateChainingHashTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SEPARATE_CHAINING_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SimpleScalarSeparateChainingHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/SimpleScalarSeparateChainingHashTable.cpp b/storage/SimpleScalarSeparateChainingHashTable.cpp
index 85e1b83..5c82d1b 100644
--- a/storage/SimpleScalarSeparateChainingHashTable.cpp
+++ b/storage/SimpleScalarSeparateChainingHashTable.cpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/SimpleScalarSeparateChainingHashTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SimpleScalarSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/SimpleScalarSeparateChainingHashTable.hpp b/storage/SimpleScalarSeparateChainingHashTable.hpp
index 962a66c..8448896 100644
--- a/storage/SimpleScalarSeparateChainingHashTable.hpp
+++ b/storage/SimpleScalarSeparateChainingHashTable.hpp
@@ -1,17 +1,20 @@
 /**
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SIMPLE_SCALAR_SEPARATE_CHAINING_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SplitRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.cpp b/storage/SplitRowStoreTupleStorageSubBlock.cpp
index 6c70d0f..f955c99 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.cpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/SplitRowStoreTupleStorageSubBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SplitRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.hpp b/storage/SplitRowStoreTupleStorageSubBlock.hpp
index 2e46bbb..a930103 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.hpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SPLIT_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 9ea1a3a..61bb7bf 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_SPLIT_ROW_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlob.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlob.hpp b/storage/StorageBlob.hpp
index 272a51a..5774fd8 100644
--- a/storage/StorageBlob.hpp
+++ b/storage/StorageBlob.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_BLOB_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index fdd438d..21aa12c 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/StorageBlock.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 3ae3812..97b4773 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlockBase.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockBase.hpp b/storage/StorageBlockBase.hpp
index 78368ee..4759a1a 100644
--- a/storage/StorageBlockBase.hpp
+++ b/storage/StorageBlockBase.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_BLOCK_BASE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlockInfo.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.cpp b/storage/StorageBlockInfo.cpp
index 39c9d0e..8c40be6 100644
--- a/storage/StorageBlockInfo.cpp
+++ b/storage/StorageBlockInfo.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 <string>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlockInfo.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.hpp b/storage/StorageBlockInfo.hpp
index c18a8a2..8d443d5 100644
--- a/storage/StorageBlockInfo.hpp
+++ b/storage/StorageBlockInfo.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_STORAGE_STORAGE_BLOCK_INFO_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/storage/StorageBlockLayout.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.cpp b/storage/StorageBlockLayout.cpp
index e28fc55..bdeb9cb 100644
--- a/storage/StorageBlockLayout.cpp
+++ b/storage/StorageBlockLayout.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "storage/StorageBlockLayout.hpp"



[24/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Join.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Join.test b/query_optimizer/tests/resolver/Join.test
index 3443e2b..08ccd04 100644
--- a/query_optimizer/tests/resolver/Join.test
+++ b/query_optimizer/tests/resolver/Join.test
@@ -1,17 +1,19 @@
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 [default initial_logical_plan]
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Select.test b/query_optimizer/tests/resolver/Select.test
index 5e11ac0..7033dec 100644
--- a/query_optimizer/tests/resolver/Select.test
+++ b/query_optimizer/tests/resolver/Select.test
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Query identifiers are case-insensitive.
 [default initial_logical_plan]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/tests/resolver/Update.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Update.test b/query_optimizer/tests/resolver/Update.test
index c0cba33..3ec5506 100644
--- a/query_optimizer/tests/resolver/Update.test
+++ b/query_optimizer/tests/resolver/Update.test
@@ -1,17 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # The first three assignment expressions are coerced to match
 # the corresponding column's type.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 6d16930..056e76d 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/AggregationOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 5bbf2f9..31c1da4 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_AGGREGATION_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 1c2ff05..465621c 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/BuildHashOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 41346c8..4a80a8a 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_BUILD_HASH_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index a51370b..9696392 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015-2016 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 QS_PROTOBUF_GENERATE_CPP(relationaloperators_SortMergeRunOperator_proto_srcs
                          relationaloperators_SortMergeRunOperator_proto_hdrs
@@ -436,7 +436,7 @@ target_link_libraries(quickstep_relationaloperators_WindowAggregationOperator
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_WindowAggregationOperationState
                       quickstep_utility_Macros
-                      tmb)                      
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_WorkOrder
                       glog
                       quickstep_queryexecution_QueryExecutionTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/CreateIndexOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.cpp b/relational_operators/CreateIndexOperator.cpp
index ed8830b..ab3624c 100644
--- a/relational_operators/CreateIndexOperator.cpp
+++ b/relational_operators/CreateIndexOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/CreateIndexOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 4e05448..fa992c9 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_CREATE_INDEX_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/CreateTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.cpp b/relational_operators/CreateTableOperator.cpp
index 81bf11a..261bec1 100644
--- a/relational_operators/CreateTableOperator.cpp
+++ b/relational_operators/CreateTableOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/CreateTableOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 7786cef..3a2e29b 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_CREATE_TABLE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 933918b..8197aef 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/DeleteOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 6bb2075..835434a 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_DELETE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index e748470..96d2561 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/DestroyHashOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index fc48ef9..d402766 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_DESTROY_HASH_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index 727aa46..5cd5ebc 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/DropTableOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index ab3344d..e713bd6 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_DROP_TABLE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 20d0ee5..65e62c4 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/FinalizeAggregationOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index af11bc3..7ac6712 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_FINALIZE_AGGREGATION_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 667df1e..7851f41 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/HashJoinOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 235bfe4..fa393b6 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_HASH_JOIN_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/InsertOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.cpp b/relational_operators/InsertOperator.cpp
index 963cdcd..31c7fa8 100644
--- a/relational_operators/InsertOperator.cpp
+++ b/relational_operators/InsertOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/InsertOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index bf9c56a..fcde663 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_INSERT_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 43588ee..9780c91 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/NestedLoopsJoinOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 041b8e9..3e3b049 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_NESTED_LOOPS_JOIN_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 3125447..fe4be68 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_REBUILD_WORKORDER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index b8d1bd0..f0303e5 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_RELATIONAL_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/RelationalOperatorsModule.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperatorsModule.hpp b/relational_operators/RelationalOperatorsModule.hpp
index 094c9a7..424a153 100644
--- a/relational_operators/RelationalOperatorsModule.hpp
+++ b/relational_operators/RelationalOperatorsModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup RelationalOperators

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index 5e5a417..63733bf 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/SampleOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 400a83f..d18b9a8 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_SAMPLE_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 5e0f33d..04d4211 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/SaveBlocksOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index d56ee2c..e84b641 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_SAVE_BLOCKS_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index eb6277e..d56326e 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/SelectOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 764dfa3..0f5c712 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_SELECT_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 1603b78..7de17ab 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/SortMergeRunOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index 531e269..0839320 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_SORT_MERGE_RUN_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortMergeRunOperator.proto
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.proto b/relational_operators/SortMergeRunOperator.proto
index 3b8a777..442b826 100644
--- a/relational_operators/SortMergeRunOperator.proto
+++ b/relational_operators/SortMergeRunOperator.proto
@@ -1,17 +1,19 @@
-//   Copyright 2011-2015 Quickstep Technologies LLC.
-//   Copyright 2015-2016 Pivotal Software, Inc.
+// 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
 //
-//   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
 //
-//       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.
+// 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.
 
 syntax = "proto2";
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortMergeRunOperatorHelpers.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperatorHelpers.cpp b/relational_operators/SortMergeRunOperatorHelpers.cpp
index 4b8c0d9..35923ab 100644
--- a/relational_operators/SortMergeRunOperatorHelpers.cpp
+++ b/relational_operators/SortMergeRunOperatorHelpers.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/SortMergeRunOperatorHelpers.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortMergeRunOperatorHelpers.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperatorHelpers.hpp b/relational_operators/SortMergeRunOperatorHelpers.hpp
index 37e1c94..af7159a 100644
--- a/relational_operators/SortMergeRunOperatorHelpers.hpp
+++ b/relational_operators/SortMergeRunOperatorHelpers.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_SORT_MERGE_RUN_OPERATOR_HELPERS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index d7362db..a14a1e4 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/SortRunGenerationOperator.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index d43b90b..125878c 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_RELATIONAL_OPERATORS_SORT_RUN_GENERATION_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index d5a08ec..3f62fc9 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "relational_operators/TableGeneratorOperator.hpp"


[27/50] [abbrv] incubator-quickstep git commit: QUICKSTEP-40: Fix Copyright notice to confirm to Apache.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/InsertTuple.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertTuple.cpp b/query_optimizer/physical/InsertTuple.cpp
index c49fa8a..3085389 100644
--- a/query_optimizer/physical/InsertTuple.cpp
+++ b/query_optimizer/physical/InsertTuple.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/InsertTuple.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/InsertTuple.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/InsertTuple.hpp b/query_optimizer/physical/InsertTuple.hpp
index 1c1799e..40f2582 100644
--- a/query_optimizer/physical/InsertTuple.hpp
+++ b/query_optimizer/physical/InsertTuple.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_INSERT_TUPLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Join.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Join.hpp b/query_optimizer/physical/Join.hpp
index d583dbd..305aa52 100644
--- a/query_optimizer/physical/Join.hpp
+++ b/query_optimizer/physical/Join.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/NestedLoopsJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/NestedLoopsJoin.cpp b/query_optimizer/physical/NestedLoopsJoin.cpp
index 9ce2077..ba3d223 100644
--- a/query_optimizer/physical/NestedLoopsJoin.cpp
+++ b/query_optimizer/physical/NestedLoopsJoin.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/NestedLoopsJoin.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/NestedLoopsJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/NestedLoopsJoin.hpp b/query_optimizer/physical/NestedLoopsJoin.hpp
index 0f9350c..c547ae8 100644
--- a/query_optimizer/physical/NestedLoopsJoin.hpp
+++ b/query_optimizer/physical/NestedLoopsJoin.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_NESTED_LOOPS_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/OptimizerPhysicalModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/OptimizerPhysicalModule.hpp b/query_optimizer/physical/OptimizerPhysicalModule.hpp
index 2fef87e..fd2cc02 100644
--- a/query_optimizer/physical/OptimizerPhysicalModule.hpp
+++ b/query_optimizer/physical/OptimizerPhysicalModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup OptimizerPhysical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PatternMatcher.hpp b/query_optimizer/physical/PatternMatcher.hpp
index fe932d5..5cd6fd3 100644
--- a/query_optimizer/physical/PatternMatcher.hpp
+++ b/query_optimizer/physical/PatternMatcher.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_PATTERN_MATCHER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Physical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Physical.hpp b/query_optimizer/physical/Physical.hpp
index 9fdbeb5..4bed593 100644
--- a/query_optimizer/physical/Physical.hpp
+++ b/query_optimizer/physical/Physical.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_PHYSICAL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index 14f8e1a..f5f35a1 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_PHYSICAL_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Sample.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Sample.cpp b/query_optimizer/physical/Sample.cpp
index 450f140..689c3cc 100644
--- a/query_optimizer/physical/Sample.cpp
+++ b/query_optimizer/physical/Sample.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/Sample.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Sample.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Sample.hpp b/query_optimizer/physical/Sample.hpp
index 5dfa1a0..e3e8f17 100644
--- a/query_optimizer/physical/Sample.hpp
+++ b/query_optimizer/physical/Sample.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_SAMPLE_HPP_
@@ -42,7 +44,7 @@ class Sample;
 typedef std::shared_ptr<const Sample> SamplePtr;
 
 /**
- * @brief Sample operator that samples the data at a block level or 
+ * @brief Sample operator that samples the data at a block level or
  *        at the tuple level and outputs them as a new relation.
  */
 class Sample : public Physical {
@@ -91,7 +93,7 @@ class Sample : public Physical {
    * @brief Creates a SamplePtr.
    * @param input The input node.
    * @param is_block_sample Flag indicating if the sample is block or tuple level
-   * @param percentage The percentage of rows or blocks to be sampled 
+   * @param percentage The percentage of rows or blocks to be sampled
    * @return An immutable Sample.
    */
   static SamplePtr Create(const PhysicalPtr &input,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Selection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.cpp b/query_optimizer/physical/Selection.cpp
index 5e1a03f..36ade04 100644
--- a/query_optimizer/physical/Selection.cpp
+++ b/query_optimizer/physical/Selection.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/Selection.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index d8c1319..b6874a1 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_SELECTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/SharedSubplanReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/SharedSubplanReference.cpp b/query_optimizer/physical/SharedSubplanReference.cpp
index eb33d32..1ba9b16 100644
--- a/query_optimizer/physical/SharedSubplanReference.cpp
+++ b/query_optimizer/physical/SharedSubplanReference.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/SharedSubplanReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/SharedSubplanReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/SharedSubplanReference.hpp b/query_optimizer/physical/SharedSubplanReference.hpp
index 4439256..60a2866 100644
--- a/query_optimizer/physical/SharedSubplanReference.hpp
+++ b/query_optimizer/physical/SharedSubplanReference.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_SHARED_SUBPLAN_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Sort.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Sort.cpp b/query_optimizer/physical/Sort.cpp
index 7ca29ac..b8bbbd4 100644
--- a/query_optimizer/physical/Sort.cpp
+++ b/query_optimizer/physical/Sort.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/Sort.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/Sort.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Sort.hpp b/query_optimizer/physical/Sort.hpp
index dac7b5b..d0c6fe8 100644
--- a/query_optimizer/physical/Sort.hpp
+++ b/query_optimizer/physical/Sort.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_SORT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/TableGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableGenerator.hpp b/query_optimizer/physical/TableGenerator.hpp
index 673c675..c9ff8a8 100644
--- a/query_optimizer/physical/TableGenerator.hpp
+++ b/query_optimizer/physical/TableGenerator.hpp
@@ -1,19 +1,20 @@
 /**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
- *   Copyright 2016 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_TABLE_GENERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/TableReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.cpp b/query_optimizer/physical/TableReference.cpp
index 4a66ddf..bfd6464 100644
--- a/query_optimizer/physical/TableReference.cpp
+++ b/query_optimizer/physical/TableReference.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/TableReference.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/TableReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.hpp b/query_optimizer/physical/TableReference.hpp
index bde9b97..638d73b 100644
--- a/query_optimizer/physical/TableReference.hpp
+++ b/query_optimizer/physical/TableReference.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_TABLE_REFERENCE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/TopLevelPlan.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TopLevelPlan.cpp b/query_optimizer/physical/TopLevelPlan.cpp
index 8b29111..6c80a35 100644
--- a/query_optimizer/physical/TopLevelPlan.cpp
+++ b/query_optimizer/physical/TopLevelPlan.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/TopLevelPlan.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/TopLevelPlan.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TopLevelPlan.hpp b/query_optimizer/physical/TopLevelPlan.hpp
index 5e655fc..8f07dec 100644
--- a/query_optimizer/physical/TopLevelPlan.hpp
+++ b/query_optimizer/physical/TopLevelPlan.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_TOP_LEVEL_PLAN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/UpdateTable.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/UpdateTable.cpp b/query_optimizer/physical/UpdateTable.cpp
index e271239..0c42543 100644
--- a/query_optimizer/physical/UpdateTable.cpp
+++ b/query_optimizer/physical/UpdateTable.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/UpdateTable.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/UpdateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/UpdateTable.hpp b/query_optimizer/physical/UpdateTable.hpp
index 4a5b992..1c93e17 100644
--- a/query_optimizer/physical/UpdateTable.hpp
+++ b/query_optimizer/physical/UpdateTable.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_UPDATETABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/WindowAggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/WindowAggregate.cpp b/query_optimizer/physical/WindowAggregate.cpp
index 2927107..f035ee3 100644
--- a/query_optimizer/physical/WindowAggregate.cpp
+++ b/query_optimizer/physical/WindowAggregate.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/physical/WindowAggregate.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/physical/WindowAggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/WindowAggregate.hpp b/query_optimizer/physical/WindowAggregate.hpp
index 4b17d07..06eaf63 100644
--- a/query_optimizer/physical/WindowAggregate.hpp
+++ b/query_optimizer/physical/WindowAggregate.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 9313e51..5251ccc 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 # Declare micro-libs:
 add_library(quickstep_queryoptimizer_resolver_NameResolver NameResolver.cpp NameResolver.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/resolver/NameResolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/NameResolver.cpp b/query_optimizer/resolver/NameResolver.cpp
index 9833f56..1c6bf6a 100644
--- a/query_optimizer/resolver/NameResolver.cpp
+++ b/query_optimizer/resolver/NameResolver.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/resolver/NameResolver.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/resolver/NameResolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/NameResolver.hpp b/query_optimizer/resolver/NameResolver.hpp
index 1618f4e..88154c7 100644
--- a/query_optimizer/resolver/NameResolver.hpp
+++ b/query_optimizer/resolver/NameResolver.hpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RESOLVER_NAME_RESOLVER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/resolver/QueryResolverModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/QueryResolverModule.hpp b/query_optimizer/resolver/QueryResolverModule.hpp
index 3757589..465dfe3 100644
--- a/query_optimizer/resolver/QueryResolverModule.hpp
+++ b/query_optimizer/resolver/QueryResolverModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup QueryResolver

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 46808bf..4931629 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/resolver/Resolver.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index 373430c..7bfeed2 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RESOLVER_RESOLVER_HPP_
@@ -499,7 +501,7 @@ class Resolver {
 
   /**
    * @brief Resolves a table/scalar subquery expression.
-   * 
+   *
    * @param parse_subquery_expression The parsed subquery expression.
    * @param type_hints The type hints for output columns by the subquery.
    * @param expression_resolution_info Resolution info that contains the name

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/BottomUpRule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/BottomUpRule.hpp b/query_optimizer/rules/BottomUpRule.hpp
index f98dadf..53dff0d 100644
--- a/query_optimizer/rules/BottomUpRule.hpp
+++ b/query_optimizer/rules/BottomUpRule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_BOTTOM_UP_RULE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 04a9814..d9709ce 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -1,19 +1,19 @@
-#   Copyright 2011-2015 Quickstep Technologies LLC.
-#   Copyright 2015 Pivotal Software, Inc.
-#   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
-#     University of Wisconsin\u2014Madison.
+# 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
 #
-#   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
 #
-#       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.
+# 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.
 
 add_subdirectory(tests)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/CollapseProject.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CollapseProject.cpp b/query_optimizer/rules/CollapseProject.cpp
index 6f2fa13..31aba17 100644
--- a/query_optimizer/rules/CollapseProject.cpp
+++ b/query_optimizer/rules/CollapseProject.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/CollapseProject.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/CollapseProject.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CollapseProject.hpp b/query_optimizer/rules/CollapseProject.hpp
index db45864..bd1a991 100644
--- a/query_optimizer/rules/CollapseProject.hpp
+++ b/query_optimizer/rules/CollapseProject.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_COLLAPSE_PROJECT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/GenerateJoins.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/GenerateJoins.cpp b/query_optimizer/rules/GenerateJoins.cpp
index 174cf03..2e5baf0 100644
--- a/query_optimizer/rules/GenerateJoins.cpp
+++ b/query_optimizer/rules/GenerateJoins.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/GenerateJoins.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/GenerateJoins.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/GenerateJoins.hpp b/query_optimizer/rules/GenerateJoins.hpp
index ecd1d13..0116e62 100644
--- a/query_optimizer/rules/GenerateJoins.hpp
+++ b/query_optimizer/rules/GenerateJoins.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_GENERATE_JOINS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/OptimizerRulesModule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/OptimizerRulesModule.hpp b/query_optimizer/rules/OptimizerRulesModule.hpp
index e6cfa1b..b96537f 100644
--- a/query_optimizer/rules/OptimizerRulesModule.hpp
+++ b/query_optimizer/rules/OptimizerRulesModule.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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.
  **/
 
 /** @defgroup OptimizerRules

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/PruneColumns.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PruneColumns.cpp b/query_optimizer/rules/PruneColumns.cpp
index 5e47b98..513ae2c 100644
--- a/query_optimizer/rules/PruneColumns.cpp
+++ b/query_optimizer/rules/PruneColumns.cpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/PruneColumns.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/PruneColumns.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PruneColumns.hpp b/query_optimizer/rules/PruneColumns.hpp
index 0188efa..5bcdced 100644
--- a/query_optimizer/rules/PruneColumns.hpp
+++ b/query_optimizer/rules/PruneColumns.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_PRUNE_COLUMNS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/PushDownFilter.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownFilter.cpp b/query_optimizer/rules/PushDownFilter.cpp
index 5bf5545..b033298 100644
--- a/query_optimizer/rules/PushDownFilter.cpp
+++ b/query_optimizer/rules/PushDownFilter.cpp
@@ -1,20 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 "query_optimizer/rules/PushDownFilter.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/918167a4/query_optimizer/rules/PushDownFilter.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownFilter.hpp b/query_optimizer/rules/PushDownFilter.hpp
index ed70d74..7d8781c 100644
--- a/query_optimizer/rules/PushDownFilter.hpp
+++ b/query_optimizer/rules/PushDownFilter.hpp
@@ -1,18 +1,20 @@
 /**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015 Pivotal Software, Inc.
+ * 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
  *
- *   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
  *
- *       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.
+ * 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 QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_FILTER_HPP_