Re: Adding a new field in SegmentsValidationAndRetentionConfig for peer segment download

2020-05-18 Thread TING CHEN
Thanks Xiaotian. My comments are inline. Some of these questions below seem
to be related to overall Deep Store by-pass design rather than this table
config change. I would provide brief answers below while directing most of
them to the cwiki design discussion.

On Mon, May 18, 2020 at 5:39 PM Xiaotian Jiang  wrote:

> Seems we always use HTTP for inner cluster communication (e.g.
> TableSizeReader).
> We should make peer download the default behavior. If the segment
> download from deep storage failed, we should always try with the peer.
> For the race condition issue, we can add CRC into the peer download
> request (CRC can be obtained from the ZK metadata).
>
> The race condition pointed out by @mcvsubbu  is
mainly for offline segment cases -- we will not address it in this design
but leave it as a TODO. When you say "peer download the default behavior"
as the default, I suppose you mean after (1) we test this feature for
realtime tables and (2) coming up with a design for offline tables and then
test on offline tables too. If this is the case, this table config change
is the right step toward that direction.

The following questions are more related to overall design:

There are some parts not clear to me:
> - Can we always obtain the deep storage address before successfully
> uploading it?
> - Should we try to at least upload once before committing the segment?
>
With a new time-bounded and best effort segment uploader added recently (PR
5314 ), the server
will wait for a configurable amount of time for the segment upload to
succeed before committing the segment. If the upload succeeds (within the
timeout period), a deep storage address will be returned.

- If we put downloadUrl before successfully uploading the segment, how
> to detect whether the upload is successful or not?
>
Similar to my answer to the previous question, with PR 5314
 we will upload the
segment before commit. I would update the cwiki about this change. The main
factor driving this change is to utilize the SplitCommitter framework.


> - When we find segment not uploaded to the deep storage, who is
> responsible of uploading it, who is responsible of updating the
> downloadUrl (controller or server)?
>
There would be follow up PR

on RealtimeValidation manager to check and let servers to upload the
segment to the deep store. The controller will update the zk data.



>
> Jackie
>
> On Mon, May 18, 2020 at 4:13 PM TING CHEN  wrote:
> >
> >
> >
> > -- Forwarded message -
> > From: TING CHEN 
> > Date: Tue, May 5, 2020 at 5:16 PM
> > Subject: Adding a new field in SegmentsValidationAndRetentionConfig for
> peer segment download
> > To: 
> >
> >
> >
> > As part of the proposal to bypass deep store for segment completion, I
> plan to add a new optional string field peerSegmentDownloadScheme to the
> SegmentsValidationAndRetentionConfig in the TableConfig. The value can be
> http or https.
> >
> > SplitSegmentCommitter  will check this value. If it exists, the segment
> committer will be able to finish segment commit successfully even if the
> upload to the segment store fails. The committer will report a special
> marker to the controller about the segment is available in peer servers.
> > When Pinot servers fail to download segments from the segment store,
> they can also check this field's value. If it exists, it can download
> segments from peer servers using either HTTP and HTTPS segment fetchers as
> configured. (related PR in review for how to discover such servers.)
> >
> > Note this is a table level config. We will test the new download
> behavior in realtime tables in incremental fashion. Once fully proven, this
> config can be upgraded to server level config.
> >
> > Please let me know if you have any questions on this. Thanks @mcvsubbu
> for coming up with the idea and offline discussions.
> >
> > Ting Chen
> >
>


Re: Adding a new field in SegmentsValidationAndRetentionConfig for peer segment download

2020-05-18 Thread TING CHEN
On Mon, May 18, 2020 at 5:24 PM Seunghyun Lee  wrote:

> If the final goal is to make this as the server level configuration (and
> remove the table level in the future), I recommend adding this config
> inside of `StreamConfig` because it will be much easier to remove the
> config from StreamConfig because it's a map.
>
> As Subbu mentioned, we may need to put it in the table config if this
> feature needs to work for both offline/realtime.
>
> @Ting Is this feature will only be used for realtime?
>

Initially yes, the peer downloading feature will only be used in real-time
ingestion. Nothing prevents offline segment loading to use peer downloading
though (with proper design overcoming a few cases raised by @mcvsubbu
) in future. So putting it in
SegmentsValidationAndRetentionConfig could be future proof -- deprecating a
config is not ideal but acceptable in this sense.

The current design does not address much on the offline segment story. But
overall, the benefits for offline tables are similar in the sense that
segments can still be downloadable when the deep store is unavailable for
some time period.



>
>
> On Mon, May 11, 2020 at 11:10 AM Subbu Subramaniam 
> wrote:
>
> > The goal of the config change in [arts other than StreamConfig is to
> > ensure that the config can be used for offline segments as well, if for
> > some reason download fails.
> >
> > However, a race condition can happen for offline segments that can be
> > dangerous. If a segment has been updated with a newer version, and
> server A
> > and B have old versions. Both of them get notified of the newer version.
> > They may try to fetch the segment and fail, and eventually fetch from
> each
> > other, and end up thinking that they have the newest version of the
> > segment.There can be other variants of this as well, with restarts of
> > server.
> >
> > I can think of some ways to fix this (e.g. in the segment update message,
> > send the crc of the new version), but these have not been fully thought
> of.
> > We need to vet these well before adopting these.
> >
> > I prefer option 1 since it introduces a single config.
> >
> > I would like to hear from @kishoreg and @npawar as well
> >
> > -Subbu
> >
> > On 2020/05/06 00:16:26, TING CHEN  wrote:
> > > As part of the proposal
> > > <
> >
> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_PINOT_By-2Dpassing-2Bdeep-2Dstore-2Brequirement-2Bfor-2BRealtime-2Bsegment-2Bcompletion&d=DwIBaQ&c=r2dcLCtU9q6n0vrtnDw9vg&r=5HWU4j1yzO0Dd4u753euNLdN_hyuGd4SHEssllc4sAY&m=2RLFABXWvcW1-9iPo8za5jQ05gjroffzi6X6Fwv193s&s=4PDasxjke_lypTn76s0wILlODFSY0ptfoHSu0cyCCxg&e=
> > >
> > > to bypass deep store for segment completion, I plan to add a new
> optional
> > > string field *peerSegmentDownloadScheme* to
> > > the SegmentsValidationAndRetentionConfig in the TableConfig. The value
> > can
> > > be *http* or *https*.
> > >
> > >1. SplitSegmentCommitter
> > ><
> >
> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_incubator-2Dpinot_blob_31c55afdb6a40f98189308ce6292587ead9d0dec_pinot-2Dcore_src_main_java_org_apache_pinot_core_data_manager_realtime_SplitSegmentCommitter.java&d=DwIBaQ&c=r2dcLCtU9q6n0vrtnDw9vg&r=5HWU4j1yzO0Dd4u753euNLdN_hyuGd4SHEssllc4sAY&m=2RLFABXWvcW1-9iPo8za5jQ05gjroffzi6X6Fwv193s&s=PwR38isPLlQEuKjFAEM6Ww3w1LD8_Goe3VbWiNSlxhc&e=
> > >
> > >will check this value. If it exists, the segment committer will be
> > able to
> > >finish segment commit successfully even if the upload to the segment
> > store
> > >fails. The committer will report a special marker to the controller
> > about
> > >the segment is available in peer servers.
> > >2. When Pinot servers fail to download segments from the segment
> > store,
> > >they can also check this field's value. If it exists, it can
> download
> > >segments from peer servers using either HTTP and HTTPS segment
> > fetchers as
> > >configured. (related PR
> > ><
> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_incubator-2Dpinot_pull_5336&d=DwIBaQ&c=r2dcLCtU9q6n0vrtnDw9vg&r=5HWU4j1yzO0Dd4u753euNLdN_hyuGd4SHEssllc4sAY&m=2RLFABXWvcW1-9iPo8za5jQ05gjroffzi6X6Fwv193s&s=vdR7nAjvQq715f8CQB2FRdCKW5vekmx5wF6D2moT2VE&e=
> > in review for
> > how
> > >to discover such servers.)
> > >
> > > Note this is a table level config. We will test the new download
> behavior
> > > in realtime tables in incremental fashion. Once fully proven, this
> config
> > > can be upgraded to server level config.
> > >
> > > Please let me know if you have any questions on this. Thanks @mcvsubbu
> > for
> > > coming up with the idea and offline discussions.
> > >
> > > Ting Chen
> > >
> >
> > -
> > To unsubscribe, e-mail: dev-unsubscr...@pinot.apache.org
> > For additional commands, e-mail: dev-h...@pinot.apache.org
> >
> >
>


Re: Adding a new field in SegmentsValidationAndRetentionConfig for peer segment download

2020-05-18 Thread Xiaotian Jiang
Seems we always use HTTP for inner cluster communication (e.g. TableSizeReader).
We should make peer download the default behavior. If the segment
download from deep storage failed, we should always try with the peer.
For the race condition issue, we can add CRC into the peer download
request (CRC can be obtained from the ZK metadata).

There are some parts not clear to me:
- Can we always obtain the deep storage address before successfully
uploading it?
- Should we try to at least upload once before committing the segment?
- If we put downloadUrl before successfully uploading the segment, how
to detect whether the upload is successful or not?
- When we find segment not uploaded to the deep storage, who is
responsible of uploading it, who is responsible of updating the
downloadUrl (controller or server)?

Jackie

On Mon, May 18, 2020 at 4:13 PM TING CHEN  wrote:
>
>
>
> -- Forwarded message -
> From: TING CHEN 
> Date: Tue, May 5, 2020 at 5:16 PM
> Subject: Adding a new field in SegmentsValidationAndRetentionConfig for peer 
> segment download
> To: 
>
>
>
> As part of the proposal to bypass deep store for segment completion, I plan 
> to add a new optional string field peerSegmentDownloadScheme to the 
> SegmentsValidationAndRetentionConfig in the TableConfig. The value can be 
> http or https.
>
> SplitSegmentCommitter  will check this value. If it exists, the segment 
> committer will be able to finish segment commit successfully even if the 
> upload to the segment store fails. The committer will report a special marker 
> to the controller about the segment is available in peer servers.
> When Pinot servers fail to download segments from the segment store, they can 
> also check this field's value. If it exists, it can download segments from 
> peer servers using either HTTP and HTTPS segment fetchers as configured. 
> (related PR in review for how to discover such servers.)
>
> Note this is a table level config. We will test the new download behavior in 
> realtime tables in incremental fashion. Once fully proven, this config can be 
> upgraded to server level config.
>
> Please let me know if you have any questions on this. Thanks @mcvsubbu for 
> coming up with the idea and offline discussions.
>
> Ting Chen
>

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



Re: Adding a new field in SegmentsValidationAndRetentionConfig for peer segment download

2020-05-18 Thread Seunghyun Lee
If the final goal is to make this as the server level configuration (and
remove the table level in the future), I recommend adding this config
inside of `StreamConfig` because it will be much easier to remove the
config from StreamConfig because it's a map.

As Subbu mentioned, we may need to put it in the table config if this
feature needs to work for both offline/realtime.

@Ting Is this feature will only be used for realtime?



On Mon, May 11, 2020 at 11:10 AM Subbu Subramaniam 
wrote:

> The goal of the config change in [arts other than StreamConfig is to
> ensure that the config can be used for offline segments as well, if for
> some reason download fails.
>
> However, a race condition can happen for offline segments that can be
> dangerous. If a segment has been updated with a newer version, and server A
> and B have old versions. Both of them get notified of the newer version.
> They may try to fetch the segment and fail, and eventually fetch from each
> other, and end up thinking that they have the newest version of the
> segment.There can be other variants of this as well, with restarts of
> server.
>
> I can think of some ways to fix this (e.g. in the segment update message,
> send the crc of the new version), but these have not been fully thought of.
> We need to vet these well before adopting these.
>
> I prefer option 1 since it introduces a single config.
>
> I would like to hear from @kishoreg and @npawar as well
>
> -Subbu
>
> On 2020/05/06 00:16:26, TING CHEN  wrote:
> > As part of the proposal
> > <
> https://cwiki.apache.org/confluence/display/PINOT/By-passing+deep-store+requirement+for+Realtime+segment+completion
> >
> > to bypass deep store for segment completion, I plan to add a new optional
> > string field *peerSegmentDownloadScheme* to
> > the SegmentsValidationAndRetentionConfig in the TableConfig. The value
> can
> > be *http* or *https*.
> >
> >1. SplitSegmentCommitter
> ><
> https://github.com/apache/incubator-pinot/blob/31c55afdb6a40f98189308ce6292587ead9d0dec/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SplitSegmentCommitter.java
> >
> >will check this value. If it exists, the segment committer will be
> able to
> >finish segment commit successfully even if the upload to the segment
> store
> >fails. The committer will report a special marker to the controller
> about
> >the segment is available in peer servers.
> >2. When Pinot servers fail to download segments from the segment
> store,
> >they can also check this field's value. If it exists, it can download
> >segments from peer servers using either HTTP and HTTPS segment
> fetchers as
> >configured. (related PR
> > in review for
> how
> >to discover such servers.)
> >
> > Note this is a table level config. We will test the new download behavior
> > in realtime tables in incremental fashion. Once fully proven, this config
> > can be upgraded to server level config.
> >
> > Please let me know if you have any questions on this. Thanks @mcvsubbu
> for
> > coming up with the idea and offline discussions.
> >
> > Ting Chen
> >
>
> -
> To unsubscribe, e-mail: dev-unsubscr...@pinot.apache.org
> For additional commands, e-mail: dev-h...@pinot.apache.org
>
>