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

[CARBONDATA-2417] [SDK Writer] SDK writer goes to infinite wait when consumer thread goes dead #2286

Closed

Conversation

manishgupta88
Copy link
Contributor

@manishgupta88 manishgupta88 commented May 9, 2018

Problem:
SDK Writer going into infinte loop in case of multi-threaded scenario

Analysis:
In multi-threaded scenarios where multiple instances of writer thread are trying to add a row to the RowBatch, addition to given batch size cannot be ensured as addition process is not synchronized and it can lead to ArrayIndexOutOfBound Exception or data loss/mismatch issues.
If multiple writer threads are adding the data to RowBatch and immediately after launching all the threads closeWriter is called, in that case we don't know when all the data is finished writing by all the threads but we are returning immediately from close writer after setting the close flag to true. This does not ensure complete processing of data.

Solution:
Make the row addition logic synchronized and modify the code in closeWriter to ensure data completeness.

  • Any interfaces changed?
    No

  • Any backward compatibility impacted?
    No

  • Document update required?
    No

  • Testing done
    Manually verified

  • For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
    NA

@CarbonDataQA
Copy link

Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4605/

@CarbonDataQA
Copy link

Build Failed with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5764/

@ravipesala
Copy link
Contributor

SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4825/

// writer thread are trying to add a row to the RowBatch. In those cases addition to given
// batch size cannot be ensured and it can lead to ArrayIndexOutOfBound Exception or data
// loss/mismatch issues
synchronized (lock) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Adding synchronisation for each row will hit data loading performance using output format. Current writer interface is for single thread. If you are adding synchronisation to handle multi threaded scenario but consumer is only one, so it will not help. We need to add one more api where user can pass number of threads and based on thread we can create multiple Producer(CarbonOutputIteratorWrapper.java) and pass the same consumer to all.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Even though current writer interface is for single thread we cant block its usage for multi-threaded scenario i.e write method is called by multiple threads using the same writer instance.

  1. If there is a single writer instance and only one thread calls the write interface then there is no impact on the performance as the call will come one by one from the same thread and lock will be acquired only by that thread.
  2. If there is single writer and multiple threads are calling the write interface using the same writer instance then locking is required because add a row to RowBatch is not synchronized and can lead to ArrayIndexoutOfBound or data loss/mismatch issues.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the developer SDK we are providing not end-user interface. The developer needs to follow as per the java doc provided by sdk. Even many Java APIs are not thread safe, and it is mentioned in there doc as well, so it is same as that. For loading multi-threaded we can add another SDK API to load faster.
In this PR you are just allowing multi-threaded while writing but it is useless if the internal dataloader does not support multi threading. So we should provide another API to allow multithread or parallel load scenarios.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kumarvishal09 @ravipesala ....I got your point. In that case synchronization changes are not required

@CarbonDataQA
Copy link

Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5789/

@CarbonDataQA
Copy link

Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4634/

@ravipesala
Copy link
Contributor

SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4842/

if (loadBatch.size > 0) {
queue.put(loadBatch);
// flag to check whether all the RowBatches have been consumed
boolean processingFinished = false;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As this class will be created for each thread as mentioned in earlier comment, i think below changes is also not required. In case of multi threaded each writer will have its own CarbonOutputIteratorWrapper object.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As discussed with you this change is required because put is a blocking call and if consumer is slow or any exception occurs then it is a possibility that the thread will still be waiting at put method for the last RowBatch

SDK Writer going into infinte loop in case of multi-threaded scenario

Analysis:
In multi-threaded scenarios where multiple instances of writer thread are trying to add a row to the RowBatch, addition to given
batch size cannot be ensured as addition process is not synchronized and it can lead to ArrayIndexOutOfBound Exception or data loss/mismatch issues
If multiple writer threads are adding the data to RowBatch and immediately after launching all the threads closeWriter is called, in that case we
don't know when all the data is finished writing by all the threads but we are returning immediately from close writer after setting the close flag
to true. This does not ensure complete processing of data.

Solution:
This API is meant for single threaded scenarios and is not thread safe. The same is updated in the interface docs.
@CarbonDataQA
Copy link

Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5805/

@CarbonDataQA
Copy link

Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4650/

@asfgit asfgit closed this in fe436c3 May 10, 2018
@ravipesala
Copy link
Contributor

SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4857/

anubhav100 pushed a commit to anubhav100/incubator-carbondata that referenced this pull request Jun 22, 2018
…consumer thread goes dead

Problem:
SDK Writer going into infinte loop in case of multi-threaded scenario

Analysis:
In multi-threaded scenarios where multiple instances of writer thread are trying to add a row to the RowBatch, addition to given batch size cannot be ensured as addition process is not synchronized and it can lead to ArrayIndexOutOfBound Exception or data loss/mismatch issues.
If multiple writer threads are adding the data to RowBatch and immediately after launching all the threads closeWriter is called, in that case we don't know when all the data is finished writing by all the threads but we are returning immediately from close writer after setting the close flag to true. This does not ensure complete processing of data.

Solution:
Make the row addition logic synchronized and modify the code in closeWriter to ensure data completeness.

This closes apache#2286
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants