Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Protobuf error on SPARK with cudf data [BUG] #10755

Closed
MikeChenfu opened this issue Apr 28, 2022 · 3 comments
Closed

Protobuf error on SPARK with cudf data [BUG] #10755

MikeChenfu opened this issue Apr 28, 2022 · 3 comments
Labels
bug Something isn't working cuIO cuIO issue

Comments

@MikeChenfu
Copy link

Describe the bug
Hello, we try using pyspark to query the data generated by cudf, but get the error InvalidProtocolBufferException: Protocol message contained an invalid tag (zero) . This error occurs at Rapids 22.02 and later version. I reproduce that on the below code. Appreciate it if anyone can help us.

Steps/Code to reproduce bug
Cudf code

import cudf
import numpy as np
import pandas as pd
import random

data1 = np.random.randint(1562167090,3062167090,size=1000000)
data2 = np.random.randint(1141541519,4269766883,size=1000000)
data3 = np.random.randint(1,100,size=1000000)

s = ['jip/Bloem-12-x-12-x-9-Casper-White-Plastic-and-Resin-Round-Center-Mounted-Rail-Planter/998151406',\
     'jsearch/?cat_id=0&query=toilet+paper', 'jip/Disney-s-Encanto-Mirabel-Floral-Toddler-And-Youth-Girls-Short-Sleeve-Graphic-T-Shirt/970697905?wmlspartner=wlpa&selectedSellerId=511', \
     None,'jip/Refurbished-Apple-iPad-Pro-12-9-5th-Gen-128GB-Space-Gray-Wi-Fi-MHNF3LL-A-Latest-Model/201301607' ] * 200000 

df = cudf.DataFrame({'s1': s, 'id1' : data1, 'id2': data2, 'id3': data3})
df['s1'][(df['s1'].str.strip() == '')] = None
df.to_orc('test.orc',index=False, compression="snappy", enable_statistics=False)

PySpark 2.4.7

from pyspark.sql.types import *
from pyspark.sql import SparkSession
from pyspark.sql.functions import lit

spark  = SparkSession.builder.master("local").config("spark.executor.memory", "70g")\
     .config("spark.driver.memory", "5g")\
     .config("spark.memory.offHeap.enabled","true")\
     .config("spark.memory.offHeap.size","32g").enableHiveSupport().getOrCreate()
sc = spark.sparkContext
df=spark.read.orc('test.orc')
df.createOrReplaceTempView("gpu_table")
print('First query')
spark.sql("select s1 from gpu_table limit 5  ").show()
print('Second query')
spark.sql("select s1 from gpu_table where id1=2627307743 and id2=1918486877 and id3=1 limit 5  ").show()

First query
+--------------------+
|                  s1|
+--------------------+
|jip/Bloem-12-x-12...|
|jsearch/?cat_id=0...|
|jip/Disney-s-Enca...|
|                null|
|jip/Refurbished-A...|
+--------------------+

Second query
---------------------------------------------------------------------------
Py4JJavaError                             Traceback (most recent call last)
<ipython-input-47-830bdf851f3c> in <module>
     72 
     73 spark.sql("select s1 from stg_csd_adobe_event_gpu  \
---> 74 where id1=2627307743 and id2=1918486877 and id3=1 limit 5  ").show()
     75 

/conda/envs/rapids/lib/python3.7/site-packages/pyspark/sql/dataframe.py in show(self, n, truncate, vertical)
    379         """
    380         if isinstance(truncate, bool) and truncate:
--> 381             print(self._jdf.showString(n, 20, vertical))
    382         else:
    383             print(self._jdf.showString(n, int(truncate), vertical))

/conda/envs/rapids/lib/python3.7/site-packages/py4j/java_gateway.py in __call__(self, *args)
   1255         answer = self.gateway_client.send_command(command)
   1256         return_value = get_return_value(
-> 1257             answer, self.gateway_client, self.target_id, self.name)
   1258 
   1259         for temp_arg in temp_args:

/conda/envs/rapids/lib/python3.7/site-packages/pyspark/sql/utils.py in deco(*a, **kw)
     61     def deco(*a, **kw):
     62         try:
---> 63             return f(*a, **kw)
     64         except py4j.protocol.Py4JJavaError as e:
     65             s = e.java_exception.toString()

/conda/envs/rapids/lib/python3.7/site-packages/py4j/protocol.py in get_return_value(answer, gateway_client, target_id, name)
    326                 raise Py4JJavaError(
    327                     "An error occurred while calling {0}{1}{2}.\n".
--> 328                     format(target_id, ".", name), value)
    329             else:
    330                 raise Py4JError(

Py4JJavaError: An error occurred while calling o2104.showString.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 125.0 failed 1 times, most recent failure: Lost task 0.0 in stage 125.0 (TID 239, localhost, executor driver): com.google.protobuf25.InvalidProtocolBufferException: Protocol message contained an invalid tag (zero).
	at com.google.protobuf25.InvalidProtocolBufferException.invalidTag(InvalidProtocolBufferException.java:89)
	at com.google.protobuf25.CodedInputStream.readTag(CodedInputStream.java:108)
	at org.apache.orc.OrcProto$RowIndexEntry.<init>(OrcProto.java:7019)
	at org.apache.orc.OrcProto$RowIndexEntry.<init>(OrcProto.java:6983)
	at org.apache.orc.OrcProto$RowIndexEntry$1.parsePartialFrom(OrcProto.java:7098)
	at org.apache.orc.OrcProto$RowIndexEntry$1.parsePartialFrom(OrcProto.java:7093)
	at com.google.protobuf25.CodedInputStream.readMessage(CodedInputStream.java:309)
	at org.apache.orc.OrcProto$RowIndex.<init>(OrcProto.java:7707)
	at org.apache.orc.OrcProto$RowIndex.<init>(OrcProto.java:7654)
	at org.apache.orc.OrcProto$RowIndex$1.parsePartialFrom(OrcProto.java:7743)
	at org.apache.orc.OrcProto$RowIndex$1.parsePartialFrom(OrcProto.java:7738)
	at com.google.protobuf25.AbstractParser.parseFrom(AbstractParser.java:89)
	at com.google.protobuf25.AbstractParser.parseFrom(AbstractParser.java:95)
	at com.google.protobuf25.AbstractParser.parseFrom(AbstractParser.java:49)
	at org.apache.orc.OrcProto$RowIndex.parseFrom(OrcProto.java:7875)
	at org.apache.orc.impl.RecordReaderUtils$DefaultDataReader.readRowIndex(RecordReaderUtils.java:232)
	at org.apache.orc.impl.RecordReaderImpl.readRowIndex(RecordReaderImpl.java:1381)
	at org.apache.orc.impl.RecordReaderImpl.readRowIndex(RecordReaderImpl.java:1364)
	at org.apache.orc.impl.RecordReaderImpl.pickRowGroups(RecordReaderImpl.java:1018)
	at org.apache.orc.impl.RecordReaderImpl.readStripe(RecordReaderImpl.java:1049)
	at org.apache.orc.impl.RecordReaderImpl.advanceStripe(RecordReaderImpl.java:1216)
	at org.apache.orc.impl.RecordReaderImpl.advanceToNextRow(RecordReaderImpl.java:1251)
	at org.apache.orc.impl.RecordReaderImpl.<init>(RecordReaderImpl.java:276)
	at org.apache.orc.impl.ReaderImpl.rows(ReaderImpl.java:639)
	at org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader.initialize(OrcColumnarBatchReader.java:140)
	at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anonfun$buildReaderWithPartitionValues$2.apply(OrcFileFormat.scala:210)
	at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anonfun$buildReaderWithPartitionValues$2.apply(OrcFileFormat.scala:174)
	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:124)
	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177)
	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.scan_nextBatch_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$13$$anon$1.hasNext(WholeStageCodegenExec.scala:636)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:858)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:858)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:123)
	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1925)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1913)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1912)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1912)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:948)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:948)
	at scala.Option.foreach(Option.scala:257)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:948)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2146)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2095)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2084)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:759)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2082)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2101)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:365)
	at org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:38)
	at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collectFromPlan(Dataset.scala:3389)
	at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2550)
	at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2550)
	at org.apache.spark.sql.Dataset$$anonfun$52.apply(Dataset.scala:3370)
	at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:80)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:127)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
	at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$withAction(Dataset.scala:3369)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:2550)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:2764)
	at org.apache.spark.sql.Dataset.getRows(Dataset.scala:254)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:291)
	at sun.reflect.GeneratedMethodAccessor136.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
	at py4j.Gateway.invoke(Gateway.java:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
Caused by: com.google.protobuf25.InvalidProtocolBufferException: Protocol message contained an invalid tag (zero).
	at com.google.protobuf25.InvalidProtocolBufferException.invalidTag(InvalidProtocolBufferException.java:89)
	at com.google.protobuf25.CodedInputStream.readTag(CodedInputStream.java:108)
	at org.apache.orc.OrcProto$RowIndexEntry.<init>(OrcProto.java:7019)
	at org.apache.orc.OrcProto$RowIndexEntry.<init>(OrcProto.java:6983)
	at org.apache.orc.OrcProto$RowIndexEntry$1.parsePartialFrom(OrcProto.java:7098)
	at org.apache.orc.OrcProto$RowIndexEntry$1.parsePartialFrom(OrcProto.java:7093)
	at com.google.protobuf25.CodedInputStream.readMessage(CodedInputStream.java:309)
	at org.apache.orc.OrcProto$RowIndex.<init>(OrcProto.java:7707)
	at org.apache.orc.OrcProto$RowIndex.<init>(OrcProto.java:7654)
	at org.apache.orc.OrcProto$RowIndex$1.parsePartialFrom(OrcProto.java:7743)
	at org.apache.orc.OrcProto$RowIndex$1.parsePartialFrom(OrcProto.java:7738)
	at com.google.protobuf25.AbstractParser.parseFrom(AbstractParser.java:89)
	at com.google.protobuf25.AbstractParser.parseFrom(AbstractParser.java:95)
	at com.google.protobuf25.AbstractParser.parseFrom(AbstractParser.java:49)
	at org.apache.orc.OrcProto$RowIndex.parseFrom(OrcProto.java:7875)
	at org.apache.orc.impl.RecordReaderUtils$DefaultDataReader.readRowIndex(RecordReaderUtils.java:232)
	at org.apache.orc.impl.RecordReaderImpl.readRowIndex(RecordReaderImpl.java:1381)
	at org.apache.orc.impl.RecordReaderImpl.readRowIndex(RecordReaderImpl.java:1364)
	at org.apache.orc.impl.RecordReaderImpl.pickRowGroups(RecordReaderImpl.java:1018)
	at org.apache.orc.impl.RecordReaderImpl.readStripe(RecordReaderImpl.java:1049)
	at org.apache.orc.impl.RecordReaderImpl.advanceStripe(RecordReaderImpl.java:1216)
	at org.apache.orc.impl.RecordReaderImpl.advanceToNextRow(RecordReaderImpl.java:1251)
	at org.apache.orc.impl.RecordReaderImpl.<init>(RecordReaderImpl.java:276)
	at org.apache.orc.impl.ReaderImpl.rows(ReaderImpl.java:639)
	at org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader.initialize(OrcColumnarBatchReader.java:140)
	at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anonfun$buildReaderWithPartitionValues$2.apply(OrcFileFormat.scala:210)
	at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anonfun$buildReaderWithPartitionValues$2.apply(OrcFileFormat.scala:174)
	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:124)
	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177)
	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.scan_nextBatch_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$13$$anon$1.hasNext(WholeStageCodegenExec.scala:636)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:858)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:858)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:123)
	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more


Rapids version

(rapids-22.06) root@1b0a4e80eba3:~/spark# conda list cuda*
# packages in environment at /conda/envs/rapids-22.06:
#
# Name                    Version                   Build  Channel
cuda-python               11.6.1           py38h3fd9d12_0    nvidia
cudatoolkit               11.5.1               hcf5317a_9    nvidia
cudf                      22.06.00a220427 cuda_11_py38_gdc1435ba92_193    rapidsai-nightly
cudf_kafka                22.06.00a220427 py38_gdc1435ba92_193    rapidsai-nightly
dask-cuda                 22.06.00a220427         py38_23    rapidsai-nightly
dask-cudf                 22.06.00a220427 cuda_11_py38_gdc1435ba92_193    rapidsai-nightly
libcudf                   22.06.00a220427 cuda11_gdc1435ba92_193    rapidsai-nightly
libcudf_kafka             22.06.00a220427 gdc1435ba92_193    rapidsai-nightly
@MikeChenfu MikeChenfu added Needs Triage Need team to review and classify bug Something isn't working labels Apr 28, 2022
@randerzander randerzander added cuIO cuIO issue and removed Needs Triage Need team to review and classify labels May 2, 2022
@vuule
Copy link
Contributor

vuule commented May 3, 2022

Likely culprit: #10041
Merged in 22.02, adds row group stats, which are not read by cuDF.

@vuule
Copy link
Contributor

vuule commented May 3, 2022

Got a minimal-ish repro where reading the file fails with cuDF:

  num_rows = 2001*5

  for i in range(10):
    s = ['jip/Bloem-12-x-12-x-9-Casper-White-Plastic-and-Resin-Round-Center-Mounted-Rail-Planter/998151406',\
         'jsearch/?cat_id=0&query=toilet+paper', 'jip/Disney-s-Encanto-Mirabel-Floral-Toddler-And-Youth-Girls-Short-Sleeve-Graphic-T-Shirt/970697905?wmlspartner=wlpa&selectedSellerId=511', \
         None,'jip/Refurbished-Apple-iPad-Pro-12-9-5th-Gen-128GB-Space-Gray-Wi-Fi-MHNF3LL-A-Latest-Model/201301607' ] * (num_rows//5) 

    df = cudf.DataFrame({'s1': s})
    df.to_orc('test.orc')
    got = cudf.read_orc('test.orc')
    assert_frame_equal(df, got)

Everything works when there's no more than 10K rows. Otherwise, all values in the second row group are equal.
No repro when:

  • All values are valid;
  • Shorter strings;
  • Reading with Pandas.

All this points to issues with the row index (not used by Pandas).
Will investigate further when I have the bandwidth.

rapids-bot bot pushed a commit that referenced this issue May 27, 2022
Issue #10755

Fixes an issue in protobuf writer where the length on the row index entry was being written into a single byte. This would cause errors when the size is larger than 127.
The issue was uncovered when row group statistics were added. String statistics contain copies to min/max strings, so the size is unbounded.
This PR changes the protobuf writer to write the entry size as a generic uint, allowing larger entries.
Also fixed `start_row` in row group info array in the reader (unrelated).

Authors:
  - Vukasin Milovanovic (https://github.com/vuule)

Approvers:
  - Ram (Ramakrishna Prabhu) (https://github.com/rgsl888prabhu)
  - David Wendt (https://github.com/davidwendt)
  - GALI PREM SAGAR (https://github.com/galipremsagar)

URL: #10989
@randerzander
Copy link
Contributor

Closed by #10989

vuule added a commit to vuule/cudf that referenced this issue May 31, 2022
Issue rapidsai#10755

Fixes an issue in protobuf writer where the length on the row index entry was being written into a single byte. This would cause errors when the size is larger than 127.
The issue was uncovered when row group statistics were added. String statistics contain copies to min/max strings, so the size is unbounded.
This PR changes the protobuf writer to write the entry size as a generic uint, allowing larger entries.
Also fixed `start_row` in row group info array in the reader (unrelated).

Authors:
  - Vukasin Milovanovic (https://github.com/vuule)

Approvers:
  - Ram (Ramakrishna Prabhu) (https://github.com/rgsl888prabhu)
  - David Wendt (https://github.com/davidwendt)
  - GALI PREM SAGAR (https://github.com/galipremsagar)

URL: rapidsai#10989
ajschmidt8 pushed a commit that referenced this issue May 31, 2022
Issue #10755

Backporting the fix to 22.06

Fixes an issue in protobuf writer where the length on the row index entry was being written into a single byte. This would cause errors when the size is larger than 127.
The issue was uncovered when row group statistics were added. String statistics contain copies to min/max strings, so the size is unbounded.
This PR changes the protobuf writer to write the entry size as a generic uint, allowing larger entries.
Also fixed `start_row` in row group info array in the reader (unrelated).

Authors:
   - Vukasin Milovanovic (https://github.com/vuule)

Approvers:
   - GALI PREM SAGAR (https://github.com/galipremsagar)
   - AJ Schmidt (https://github.com/ajschmidt8)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working cuIO cuIO issue
Projects
None yet
Development

No branches or pull requests

3 participants