[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-28 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/21570


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-27 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r198710399
  
--- Diff: 
core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java ---
@@ -43,6 +47,12 @@ void free(long size) {
 used -= size;
 taskMemoryManager.releaseExecutionMemory(size, this);
   }
+
+  @VisibleForTesting
--- End diff --

it's already in the test package, we don't need this tag.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-26 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r198224083
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
--- End diff --

I am sorry. Nvm, I thought lines 21 and 23 are duplicated.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-26 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r198177547
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
--- End diff --

sorry but which duplicated line do you mention? 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-21 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r197328626
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
--- End diff --

cc @jiangxb1987


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-19 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r196437321
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the RecordBinaryComparator, which compares two UnsafeRows by their 
binary form.
+ */
+public class RecordBinaryComparatorSuite {
+
+  private final TaskMemoryManager memoryManager = new TaskMemoryManager(
--- End diff --

Both way shall work. I choose to implement it in low level so the result 
won't be affected by the UnsafeProjection code.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-17 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195922982
  
--- Diff: 
core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java ---
@@ -43,6 +45,12 @@ void free(long size) {
 used -= size;
 taskMemoryManager.releaseExecutionMemory(size, this);
   }
+
+  // Exposed for testing
--- End diff --

How about using `@VisibleForTesting` annotation?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-15 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195891897
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the RecordBinaryComparator, which compares two UnsafeRows by their 
binary form.
+ */
+public class RecordBinaryComparatorSuite {
+
+  private final TaskMemoryManager memoryManager = new TaskMemoryManager(
--- End diff --

We need to use `TaskMemoryManager` for this test? Is not simple tests 
(e.g., just comparing unsafe rows) enough?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-15 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195891774
  
--- Diff: 
core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java ---
@@ -43,6 +45,12 @@ void free(long size) {
 used -= size;
 taskMemoryManager.releaseExecutionMemory(size, this);
   }
+
+  // Exposed for testing
--- End diff --

super nit: we need this comment? This class itself is used for test use 
only?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-15 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195891644
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
--- End diff --

super nit: better to merge imports into one.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-15 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195640246
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
--- End diff --

nit: is it better to insert a line?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-15 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195640161
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
--- End diff --

Since this is duplicated, it should be deleted.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-15 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195639885
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the RecordBinaryComparator, which compares two UnsafeRows by their 
binary form.
+ */
+public class RecordBinaryComparatorSuite {
+
+  private final TaskMemoryManager memoryManager = new TaskMemoryManager(
+  new TestMemoryManager(new 
SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+  private final TestMemoryConsumer consumer = new 
TestMemoryConsumer(memoryManager);
+
+  private final int uaoSize = UnsafeAlignedOffset.getUaoSize();
+
+  private MemoryBlock dataPage;
+  private long pageCursor;
+
+  private LongArray array;
+  private int pos;
+
+  @Before
+  public void beforeEach() {
+// Only compare between two input rows.
+array = consumer.allocateArray(2);
+pos = 0;
+
+dataPage = memoryManager.allocatePage(4096, consumer);
+pageCursor = dataPage.getBaseOffset();
+  }
+
+  @After
+  public void afterEach() {
+consumer.freePage(dataPage);
+dataPage = null;
+pageCursor = 0;
+
+consumer.freeArray(array);
+array = null;
+pos = 0;
+  }
+
+  private void insertRow(UnsafeRow row) {
+Object recordBase = row.getBaseObject();
+long recordOffset = row.getBaseOffset();
+int recordLength = row.getSizeInBytes();
+
+Object baseObject = dataPage.getBaseObject();
+assert(pageCursor + recordLength <= dataPage.getBaseOffset() + 
dataPage.size());
+long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, 
pageCursor);
+UnsafeAlignedOffset.putSize(baseObject, pageCursor, recordLength);
+pageCursor += uaoSize;
+Platform.copyMemory(recordBase, recordOffset, baseObject, pageCursor, 
recordLength);
+pageCursor += recordLength;
+
+assert(pos < 2);
+array.set(pos, recordAddress);
+pos++;
+  }
+
+  private int compare(int index1, int index2) {
+Object baseObject = dataPage.getBaseObject();
+
+long recordAddress1 = array.get(index1);
+long baseOffset1 = memoryManager.getOffsetInPage(recordAddress1) + 
uaoSize;
+int recordLength1 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset1 - uaoSize);
+
+long recordAddress2 = array.get(index2);
+long baseOffset2 = memoryManager.getOffsetInPage(recordAddress2) + 
uaoSize;
+int recordLength2 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset2 - uaoSize);
+
+return binaryComparator.compare(baseObject, baseOffset1, 
recordLength1, baseObject,
+baseOffset2, recordLength2);
+  }
+
+  private final RecordComparator binaryComparator = new 
RecordBinaryComparator();
+
+  // Compute the most compact size for UnsafeRow's backing data.
+  private int computeSizeInBytes(int originalSize) {
+// All the 

[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195636719
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the RecordBinaryComparator, which compares two UnsafeRows by their 
binary form.
+ */
+public class RecordBinaryComparatorSuite {
+
+  private final TaskMemoryManager memoryManager = new TaskMemoryManager(
+  new TestMemoryManager(new 
SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+  private final TestMemoryConsumer consumer = new 
TestMemoryConsumer(memoryManager);
+
+  private final int uaoSize = UnsafeAlignedOffset.getUaoSize();
+
+  private MemoryBlock dataPage;
+  private long pageCursor;
+
+  private LongArray array;
+  private int pos;
+
+  @Before
+  public void beforeEach() {
+// Only compare between two input rows.
+array = consumer.allocateArray(2);
+pos = 0;
+
+dataPage = memoryManager.allocatePage(4096, consumer);
+pageCursor = dataPage.getBaseOffset();
+  }
+
+  @After
+  public void afterEach() {
+consumer.freePage(dataPage);
+dataPage = null;
+pageCursor = 0;
+
+consumer.freeArray(array);
+array = null;
+pos = 0;
+  }
+
+  private void insertRow(UnsafeRow row) {
+Object recordBase = row.getBaseObject();
+long recordOffset = row.getBaseOffset();
+int recordLength = row.getSizeInBytes();
+
+Object baseObject = dataPage.getBaseObject();
+assert(pageCursor + recordLength <= dataPage.getBaseOffset() + 
dataPage.size());
+long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, 
pageCursor);
+UnsafeAlignedOffset.putSize(baseObject, pageCursor, recordLength);
+pageCursor += uaoSize;
+Platform.copyMemory(recordBase, recordOffset, baseObject, pageCursor, 
recordLength);
+pageCursor += recordLength;
+
+assert(pos < 2);
+array.set(pos, recordAddress);
+pos++;
+  }
+
+  private int compare(int index1, int index2) {
+Object baseObject = dataPage.getBaseObject();
+
+long recordAddress1 = array.get(index1);
+long baseOffset1 = memoryManager.getOffsetInPage(recordAddress1) + 
uaoSize;
+int recordLength1 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset1 - uaoSize);
+
+long recordAddress2 = array.get(index2);
+long baseOffset2 = memoryManager.getOffsetInPage(recordAddress2) + 
uaoSize;
+int recordLength2 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset2 - uaoSize);
+
+return binaryComparator.compare(baseObject, baseOffset1, 
recordLength1, baseObject,
+baseOffset2, recordLength2);
+  }
+
+  private final RecordComparator binaryComparator = new 
RecordBinaryComparator();
+
+  // Compute the most compact size for UnsafeRow's backing data.
+  private int computeSizeInBytes(int originalSize) {
+// All the 

[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195636548
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the RecordBinaryComparator, which compares two UnsafeRows by their 
binary form.
+ */
+public class RecordBinaryComparatorSuite {
+
+  private final TaskMemoryManager memoryManager = new TaskMemoryManager(
+  new TestMemoryManager(new 
SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+  private final TestMemoryConsumer consumer = new 
TestMemoryConsumer(memoryManager);
+
+  private final int uaoSize = UnsafeAlignedOffset.getUaoSize();
+
+  private MemoryBlock dataPage;
+  private long pageCursor;
+
+  private LongArray array;
+  private int pos;
+
+  @Before
+  public void beforeEach() {
+// Only compare between two input rows.
+array = consumer.allocateArray(2);
+pos = 0;
+
+dataPage = memoryManager.allocatePage(4096, consumer);
+pageCursor = dataPage.getBaseOffset();
+  }
+
+  @After
+  public void afterEach() {
+consumer.freePage(dataPage);
+dataPage = null;
+pageCursor = 0;
+
+consumer.freeArray(array);
+array = null;
+pos = 0;
+  }
+
+  private void insertRow(UnsafeRow row) {
+Object recordBase = row.getBaseObject();
+long recordOffset = row.getBaseOffset();
+int recordLength = row.getSizeInBytes();
+
+Object baseObject = dataPage.getBaseObject();
+assert(pageCursor + recordLength <= dataPage.getBaseOffset() + 
dataPage.size());
+long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, 
pageCursor);
+UnsafeAlignedOffset.putSize(baseObject, pageCursor, recordLength);
+pageCursor += uaoSize;
+Platform.copyMemory(recordBase, recordOffset, baseObject, pageCursor, 
recordLength);
+pageCursor += recordLength;
+
+assert(pos < 2);
+array.set(pos, recordAddress);
+pos++;
+  }
+
+  private int compare(int index1, int index2) {
+Object baseObject = dataPage.getBaseObject();
+
+long recordAddress1 = array.get(index1);
+long baseOffset1 = memoryManager.getOffsetInPage(recordAddress1) + 
uaoSize;
+int recordLength1 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset1 - uaoSize);
+
+long recordAddress2 = array.get(index2);
+long baseOffset2 = memoryManager.getOffsetInPage(recordAddress2) + 
uaoSize;
+int recordLength2 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset2 - uaoSize);
+
+return binaryComparator.compare(baseObject, baseOffset1, 
recordLength1, baseObject,
+baseOffset2, recordLength2);
+  }
+
+  private final RecordComparator binaryComparator = new 
RecordBinaryComparator();
+
+  // Compute the most compact size for UnsafeRow's backing data.
+  private int computeSizeInBytes(int originalSize) {
+// All the 

[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-14 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21570#discussion_r195634729
  
--- Diff: 
sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java
 ---
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 test.org.apache.spark.sql.execution.sort;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.memory.TestMemoryConsumer;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.execution.RecordBinaryComparator;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.collection.unsafe.sort.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the RecordBinaryComparator, which compares two UnsafeRows by their 
binary form.
+ */
+public class RecordBinaryComparatorSuite {
+
+  private final TaskMemoryManager memoryManager = new TaskMemoryManager(
+  new TestMemoryManager(new 
SparkConf().set("spark.memory.offHeap.enabled", "false")), 0);
+  private final TestMemoryConsumer consumer = new 
TestMemoryConsumer(memoryManager);
+
+  private final int uaoSize = UnsafeAlignedOffset.getUaoSize();
+
+  private MemoryBlock dataPage;
+  private long pageCursor;
+
+  private LongArray array;
+  private int pos;
+
+  @Before
+  public void beforeEach() {
+// Only compare between two input rows.
+array = consumer.allocateArray(2);
+pos = 0;
+
+dataPage = memoryManager.allocatePage(4096, consumer);
+pageCursor = dataPage.getBaseOffset();
+  }
+
+  @After
+  public void afterEach() {
+consumer.freePage(dataPage);
+dataPage = null;
+pageCursor = 0;
+
+consumer.freeArray(array);
+array = null;
+pos = 0;
+  }
+
+  private void insertRow(UnsafeRow row) {
+Object recordBase = row.getBaseObject();
+long recordOffset = row.getBaseOffset();
+int recordLength = row.getSizeInBytes();
+
+Object baseObject = dataPage.getBaseObject();
+assert(pageCursor + recordLength <= dataPage.getBaseOffset() + 
dataPage.size());
+long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, 
pageCursor);
+UnsafeAlignedOffset.putSize(baseObject, pageCursor, recordLength);
+pageCursor += uaoSize;
+Platform.copyMemory(recordBase, recordOffset, baseObject, pageCursor, 
recordLength);
+pageCursor += recordLength;
+
+assert(pos < 2);
+array.set(pos, recordAddress);
+pos++;
+  }
+
+  private int compare(int index1, int index2) {
+Object baseObject = dataPage.getBaseObject();
+
+long recordAddress1 = array.get(index1);
+long baseOffset1 = memoryManager.getOffsetInPage(recordAddress1) + 
uaoSize;
+int recordLength1 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset1 - uaoSize);
+
+long recordAddress2 = array.get(index2);
+long baseOffset2 = memoryManager.getOffsetInPage(recordAddress2) + 
uaoSize;
+int recordLength2 = UnsafeAlignedOffset.getSize(baseObject, 
baseOffset2 - uaoSize);
+
+return binaryComparator.compare(baseObject, baseOffset1, 
recordLength1, baseObject,
+baseOffset2, recordLength2);
+  }
+
+  private final RecordComparator binaryComparator = new 
RecordBinaryComparator();
+
+  // Compute the most compact size for UnsafeRow's backing data.
+  private int computeSizeInBytes(int originalSize) {
+// All the 

[GitHub] spark pull request #21570: [SPARK-24564][TEST] Add test suite for RecordBina...

2018-06-14 Thread jiangxb1987
GitHub user jiangxb1987 opened a pull request:

https://github.com/apache/spark/pull/21570

[SPARK-24564][TEST] Add test suite for RecordBinaryComparator

## What changes were proposed in this pull request?

Add a new test suite to test RecordBinaryComparator.

## How was this patch tested?

New test suite.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jiangxb1987/spark rbc-test

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21570.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21570


commit 79508ec7a33735690b0d5f0c74168efbf804866c
Author: Xingbo Jiang 
Date:   2018-06-14T22:52:13Z

add test suite for RecordBinaryComparator




---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org