You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by Navis Ryu <na...@nexr.com> on 2014/05/16 08:08:40 UTC

Review Request 21549: Deduplicate columns appearing in both the key list and value list of ReduceSinkOperator

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/21549/
-----------------------------------------------------------

Review request for hive.


Bugs: HIVE-4867
    https://issues.apache.org/jira/browse/HIVE-4867


Repository: hive-git


Description
-------

A ReduceSinkOperator emits data in the format of keys and values. Right now, a column may appear in both the key list and value list, which result in unnecessary overhead for shuffling. 

Example:
We have a query shown below ...
{code:sql}
explain select ss_ticket_number from store_sales cluster by ss_ticket_number;
{\code}

The plan is ...
{code}
STAGE DEPENDENCIES:
  Stage-1 is a root stage
  Stage-0 is a root stage

STAGE PLANS:
  Stage: Stage-1
    Map Reduce
      Alias -> Map Operator Tree:
        store_sales 
          TableScan
            alias: store_sales
            Select Operator
              expressions:
                    expr: ss_ticket_number
                    type: int
              outputColumnNames: _col0
              Reduce Output Operator
                key expressions:
                      expr: _col0
                      type: int
                sort order: +
                Map-reduce partition columns:
                      expr: _col0
                      type: int
                tag: -1
                value expressions:
                      expr: _col0
                      type: int
      Reduce Operator Tree:
        Extract
          File Output Operator
            compressed: false
            GlobalTableId: 0
            table:
                input format: org.apache.hadoop.mapred.TextInputFormat
                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

  Stage: Stage-0
    Fetch Operator
      limit: -1

{\code}

The column 'ss_ticket_number' is in both the key list and value list of the ReduceSinkOperator. The type of ss_ticket_number is int. For this case, BinarySortableSerDe will introduce 1 byte more for every int in the key. LazyBinarySerDe will also introduce overhead when recording the length of a int. For every int, 10 bytes should be a rough estimation of the size of data emitted from the Map phase. 


Diffs
-----

  ql/src/java/org/apache/hadoop/hive/ql/Driver.java 9040d9b 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java acaca23 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java fc5864a 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java 22374b2 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java 6368548 
  ql/src/java/org/apache/hadoop/hive/ql/exec/RowSchema.java 083d574 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java 7250432 
  ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageInfo.java 22a8785 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java 6a4dc9b 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java e3e0acc 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java 86e4834 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java 719fe9f 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcCtx.java 7cf48a7 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b5cdde1 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java 78b7ca8 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java eac0edd 
  ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java f142f3e 
  ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 49eb83f 
  ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java 4175d11 
  ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java e706f52 

Diff: https://reviews.apache.org/r/21549/diff/


Testing
-------


Thanks,

Navis Ryu


Re: Review Request 21549: Deduplicate columns appearing in both the key list and value list of ReduceSinkOperator

Posted by Navis Ryu <na...@nexr.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/21549/
-----------------------------------------------------------

(Updated June 5, 2014, 3:32 a.m.)


Review request for hive.


Changes
-------

Merged HIVE-7173 into this.


Bugs: HIVE-4867
    https://issues.apache.org/jira/browse/HIVE-4867


Repository: hive-git


Description
-------

A ReduceSinkOperator emits data in the format of keys and values. Right now, a column may appear in both the key list and value list, which result in unnecessary overhead for shuffling. 

Example:
We have a query shown below ...
{code:sql}
explain select ss_ticket_number from store_sales cluster by ss_ticket_number;
{\code}

The plan is ...
{code}
STAGE DEPENDENCIES:
  Stage-1 is a root stage
  Stage-0 is a root stage

STAGE PLANS:
  Stage: Stage-1
    Map Reduce
      Alias -> Map Operator Tree:
        store_sales 
          TableScan
            alias: store_sales
            Select Operator
              expressions:
                    expr: ss_ticket_number
                    type: int
              outputColumnNames: _col0
              Reduce Output Operator
                key expressions:
                      expr: _col0
                      type: int
                sort order: +
                Map-reduce partition columns:
                      expr: _col0
                      type: int
                tag: -1
                value expressions:
                      expr: _col0
                      type: int
      Reduce Operator Tree:
        Extract
          File Output Operator
            compressed: false
            GlobalTableId: 0
            table:
                input format: org.apache.hadoop.mapred.TextInputFormat
                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

  Stage: Stage-0
    Fetch Operator
      limit: -1

{\code}

The column 'ss_ticket_number' is in both the key list and value list of the ReduceSinkOperator. The type of ss_ticket_number is int. For this case, BinarySortableSerDe will introduce 1 byte more for every int in the key. LazyBinarySerDe will also introduce overhead when recording the length of a int. For every int, 10 bytes should be a rough estimation of the size of data emitted from the Map phase. 


Diffs (updated)
-----

  ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java c8c890d 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java acaca23 
  ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java 28ffdd8 
  ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java aca7e7f 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java 46aff87 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java fc5864a 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java 22374b2 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java 6368548 
  ql/src/java/org/apache/hadoop/hive/ql/exec/RowSchema.java 083d574 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ec9dd8c 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java c56b7a3 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java ce249d1 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java b2596e9 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java f3b00d0 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java fb58253 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/UnwrapRowContainer.java PRE-CREATION 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java f1ebd99 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java e3e0acc 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java 86e4834 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java 719fe9f 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcCtx.java 7cf48a7 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b5cdde1 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java 78b7ca8 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java eac0edd 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java 4019320 
  ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java f142f3e 
  ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 28d0e1c 
  ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java cf32a08 
  ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java 63579fa 
  ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java 4175d11 

Diff: https://reviews.apache.org/r/21549/diff/


Testing
-------


Thanks,

Navis Ryu


Re: Review Request 21549: Deduplicate columns appearing in both the key list and value list of ReduceSinkOperator

Posted by Navis Ryu <na...@nexr.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/21549/
-----------------------------------------------------------

(Updated June 3, 2014, 4:50 a.m.)


Review request for hive.


Changes
-------

Fixed tez failures


Bugs: HIVE-4867
    https://issues.apache.org/jira/browse/HIVE-4867


Repository: hive-git


Description
-------

A ReduceSinkOperator emits data in the format of keys and values. Right now, a column may appear in both the key list and value list, which result in unnecessary overhead for shuffling. 

Example:
We have a query shown below ...
{code:sql}
explain select ss_ticket_number from store_sales cluster by ss_ticket_number;
{\code}

The plan is ...
{code}
STAGE DEPENDENCIES:
  Stage-1 is a root stage
  Stage-0 is a root stage

STAGE PLANS:
  Stage: Stage-1
    Map Reduce
      Alias -> Map Operator Tree:
        store_sales 
          TableScan
            alias: store_sales
            Select Operator
              expressions:
                    expr: ss_ticket_number
                    type: int
              outputColumnNames: _col0
              Reduce Output Operator
                key expressions:
                      expr: _col0
                      type: int
                sort order: +
                Map-reduce partition columns:
                      expr: _col0
                      type: int
                tag: -1
                value expressions:
                      expr: _col0
                      type: int
      Reduce Operator Tree:
        Extract
          File Output Operator
            compressed: false
            GlobalTableId: 0
            table:
                input format: org.apache.hadoop.mapred.TextInputFormat
                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

  Stage: Stage-0
    Fetch Operator
      limit: -1

{\code}

The column 'ss_ticket_number' is in both the key list and value list of the ReduceSinkOperator. The type of ss_ticket_number is int. For this case, BinarySortableSerDe will introduce 1 byte more for every int in the key. LazyBinarySerDe will also introduce overhead when recording the length of a int. For every int, 10 bytes should be a rough estimation of the size of data emitted from the Map phase. 


Diffs (updated)
-----

  ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java c8c890d 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java acaca23 
  ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java 28ffdd8 
  ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java aca7e7f 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java 46aff87 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java fc5864a 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java 22374b2 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java 6368548 
  ql/src/java/org/apache/hadoop/hive/ql/exec/RowSchema.java 083d574 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ec9dd8c 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java c56b7a3 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java ce249d1 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java b2596e9 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java f3b00d0 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java fb58253 
  ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/UnwrapRowContainer.java PRE-CREATION 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java f1ebd99 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java d42e1f7 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java e3e0acc 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java 86e4834 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java 719fe9f 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcCtx.java 7cf48a7 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b5cdde1 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java 78b7ca8 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java eac0edd 
  ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java f142f3e 
  ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 28d0e1c 
  ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java cf32a08 
  ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java 63579fa 
  ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java 4175d11 

Diff: https://reviews.apache.org/r/21549/diff/


Testing
-------


Thanks,

Navis Ryu


Re: Review Request 21549: Deduplicate columns appearing in both the key list and value list of ReduceSinkOperator

Posted by Navis Ryu <na...@nexr.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/21549/
-----------------------------------------------------------

(Updated May 29, 2014, 5:24 a.m.)


Review request for hive.


Changes
-------

source only


Bugs: HIVE-4867
    https://issues.apache.org/jira/browse/HIVE-4867


Repository: hive-git


Description
-------

A ReduceSinkOperator emits data in the format of keys and values. Right now, a column may appear in both the key list and value list, which result in unnecessary overhead for shuffling. 

Example:
We have a query shown below ...
{code:sql}
explain select ss_ticket_number from store_sales cluster by ss_ticket_number;
{\code}

The plan is ...
{code}
STAGE DEPENDENCIES:
  Stage-1 is a root stage
  Stage-0 is a root stage

STAGE PLANS:
  Stage: Stage-1
    Map Reduce
      Alias -> Map Operator Tree:
        store_sales 
          TableScan
            alias: store_sales
            Select Operator
              expressions:
                    expr: ss_ticket_number
                    type: int
              outputColumnNames: _col0
              Reduce Output Operator
                key expressions:
                      expr: _col0
                      type: int
                sort order: +
                Map-reduce partition columns:
                      expr: _col0
                      type: int
                tag: -1
                value expressions:
                      expr: _col0
                      type: int
      Reduce Operator Tree:
        Extract
          File Output Operator
            compressed: false
            GlobalTableId: 0
            table:
                input format: org.apache.hadoop.mapred.TextInputFormat
                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

  Stage: Stage-0
    Fetch Operator
      limit: -1

{\code}

The column 'ss_ticket_number' is in both the key list and value list of the ReduceSinkOperator. The type of ss_ticket_number is int. For this case, BinarySortableSerDe will introduce 1 byte more for every int in the key. LazyBinarySerDe will also introduce overhead when recording the length of a int. For every int, 10 bytes should be a rough estimation of the size of data emitted from the Map phase. 


Diffs (updated)
-----

  ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java acaca23 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java fc5864a 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java 22374b2 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java 6368548 
  ql/src/java/org/apache/hadoop/hive/ql/exec/RowSchema.java 083d574 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java 26cdc5a 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java f1ebd99 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java e3e0acc 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java 86e4834 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java 719fe9f 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcCtx.java 7cf48a7 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b5cdde1 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java 78b7ca8 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java eac0edd 
  ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java f142f3e 
  ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 28d0e1c 
  ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java 4175d11 

Diff: https://reviews.apache.org/r/21549/diff/


Testing
-------


Thanks,

Navis Ryu


Re: Review Request 21549: Deduplicate columns appearing in both the key list and value list of ReduceSinkOperator

Posted by Navis Ryu <na...@nexr.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/21549/
-----------------------------------------------------------

(Updated May 16, 2014, 6:08 a.m.)


Review request for hive.


Bugs: HIVE-4867
    https://issues.apache.org/jira/browse/HIVE-4867


Repository: hive-git


Description
-------

A ReduceSinkOperator emits data in the format of keys and values. Right now, a column may appear in both the key list and value list, which result in unnecessary overhead for shuffling. 

Example:
We have a query shown below ...
{code:sql}
explain select ss_ticket_number from store_sales cluster by ss_ticket_number;
{\code}

The plan is ...
{code}
STAGE DEPENDENCIES:
  Stage-1 is a root stage
  Stage-0 is a root stage

STAGE PLANS:
  Stage: Stage-1
    Map Reduce
      Alias -> Map Operator Tree:
        store_sales 
          TableScan
            alias: store_sales
            Select Operator
              expressions:
                    expr: ss_ticket_number
                    type: int
              outputColumnNames: _col0
              Reduce Output Operator
                key expressions:
                      expr: _col0
                      type: int
                sort order: +
                Map-reduce partition columns:
                      expr: _col0
                      type: int
                tag: -1
                value expressions:
                      expr: _col0
                      type: int
      Reduce Operator Tree:
        Extract
          File Output Operator
            compressed: false
            GlobalTableId: 0
            table:
                input format: org.apache.hadoop.mapred.TextInputFormat
                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

  Stage: Stage-0
    Fetch Operator
      limit: -1

{\code}

The column 'ss_ticket_number' is in both the key list and value list of the ReduceSinkOperator. The type of ss_ticket_number is int. For this case, BinarySortableSerDe will introduce 1 byte more for every int in the key. LazyBinarySerDe will also introduce overhead when recording the length of a int. For every int, 10 bytes should be a rough estimation of the size of data emitted from the Map phase. 


Diffs
-----

  ql/src/java/org/apache/hadoop/hive/ql/Driver.java 9040d9b 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java acaca23 
  ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java fc5864a 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java 22374b2 
  ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java 6368548 
  ql/src/java/org/apache/hadoop/hive/ql/exec/RowSchema.java 083d574 
  ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java 7250432 
  ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageInfo.java 22a8785 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java 6a4dc9b 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java e3e0acc 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java 86e4834 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java 719fe9f 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcCtx.java 7cf48a7 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b5cdde1 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java 78b7ca8 
  ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java eac0edd 
  ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java f142f3e 
  ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 49eb83f 
  ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java 4175d11 
  ql/src/java/org/apache/hadoop/hive/ql/session/LineageState.java e706f52 

Diff: https://reviews.apache.org/r/21549/diff/


Testing
-------


Thanks,

Navis Ryu