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

Introduce SystemSchema tables (#5989) #6094

Merged
merged 61 commits into from
Oct 11, 2018
Merged

Conversation

surekhasaharan
Copy link

@surekhasaharan surekhasaharan commented Aug 1, 2018

SYSTEM_TABLES provide visibility into the druid segments, servers and tasks.

  • SEGMENTS table provides details on served and published segments
  • SERVERS table provides details on data servers
  • SERVERSEGMETS table is the JOIN of SEGMENTS and SERVERS
  • TASKS table provides details on tasks from overlord

Sample sql queries to retrieve data from these tables:

SELECT * FROM sys.segments where datasource='wikipedia';
SELECT * FROM sys.servers;
SELECT * FROM sys.tasks where status='FAILED';

Surekha Saharan added 5 commits July 31, 2018 17:32
* SEGMENTS table provides details on served and published segments
* SERVERS table provides details on data servers
* SERVERSEGMETS table is the JOIN of SEGMENTS and SERVERS
* TASKS table provides details on tasks
@vogievetsky
Copy link
Contributor

so excited about this one!

@fjy fjy added this to the 0.13.0 milestone Aug 13, 2018
@fjy
Copy link
Contributor

fjy commented Aug 13, 2018

@surekhasaharan can you fix the merge conflicts?

@@ -481,6 +485,77 @@ SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'druid' AND TABLE_
|COLLATION_NAME||
|JDBC_TYPE|Type code from java.sql.Types (Druid extension)|

## SYSTEM SCHEMA

SYSTEM_TABLES provide visibility into the druid segments, servers and tasks.
Copy link
Contributor

Choose a reason for hiding this comment

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

Please correct capitalization and naming:

The SYS schema provides visibility into Druid segments, servers and tasks.

Copy link
Author

Choose a reason for hiding this comment

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

done

SYSTEM_TABLES provide visibility into the druid segments, servers and tasks.
For example to retrieve all segments for datasource "wikipedia", use the query:
```sql
select * from SYS.SEGMENTS where DATASOURCE='wikipedia';
Copy link
Contributor

Choose a reason for hiding this comment

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

Lowercase seems more Druid-y, so I think I'd prefer SELECT * FROM sys.segments WHERE dataSource = 'wikipedia'. The only reason INFORMATION_SCHEMA isn't like this is because it's a standard thing and uppercase seems more normal for it from looking at other databases.

Copy link
Author

Choose a reason for hiding this comment

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

ok, will make everything lowercase in docs and code. For the column names, do they need to be camelCase like dataSource, isPublished etc. or datasource, is_published or keeping them uppercase is fine ?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, good question. I think in SQL underscores are more normal, although data_source is very weird so let's not do that. Probably datasource is ok.

If anyone else has an opinion please go for it.

```

### SEGMENTS table
Segments tables provides details on all the segments, both published and served(but not published).
Copy link
Contributor

Choose a reason for hiding this comment

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

To me this reads a bit unclear, I'd suggest trying something like:

Segments tables provides details on all Druid segments, whether they are published yet or not.

Copy link
Author

Choose a reason for hiding this comment

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

changed


|Column|Notes|
|------|-----|
|SEGMENT_ID||
Copy link
Contributor

Choose a reason for hiding this comment

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

Please include a description for all of these columns, and capitalize the first letter of each description.

Copy link
Contributor

Choose a reason for hiding this comment

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

Please include "size", "version", and "partition_num" too -- they are all useful. I'd also include "replicas" which should be the number of replicas currently being served.

Copy link
Author

Choose a reason for hiding this comment

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

added

|DATASOURCE||
|START||
|END||
|IS_PUBLISHED|segment in metadata store|
Copy link
Contributor

Choose a reason for hiding this comment

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

It'd be clearer to expand this a bit: "True if this segment has been published to the metadata store."

Similar comment for the other ones.

Copy link
Author

Choose a reason for hiding this comment

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

added more description

row[1] = segment.getDataSource();
row[2] = segment.getInterval().getStart();
row[3] = segment.getInterval().getEnd();
boolean is_available = false;
Copy link
Contributor

Choose a reason for hiding this comment

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

Avoid underscores in variable names: this should be isAvailable. Similar comment for other names.

Copy link
Author

Choose a reason for hiding this comment

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

done

return Linq4j.asEnumerable(rows);
}

private List<ImmutableDruidDataSource> getMetadataSegments(
Copy link
Contributor

Choose a reason for hiding this comment

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

We should document that the coordinator must be online in order for querying this table to work properly. It's an important note since the other tables only require broker & data servers. Let's do a similar note for tasks / overlord too.

Copy link
Author

Choose a reason for hiding this comment

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

added a comment for segments and tasks table

}

private List<ImmutableDruidDataSource> getMetadataSegments(
DruidLeaderClient druidLeaderClient,
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's name this appropriately as to what it is (coordinator client).

Copy link
Author

Choose a reason for hiding this comment

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

renamed

for (DataSegment segment : segmentMap.values()) {
row[0] = druidServer.getHost();
row[1] = segment.getIdentifier();
rows.add(row);
Copy link
Contributor

Choose a reason for hiding this comment

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

This has the same issue as the segments table above. It doesn't work to reuse arrays like this.

Copy link
Author

Choose a reason for hiding this comment

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

moved Object[] row = new Object[SERVERSEGMENTS_TABLE_SIZE]; inside the inner loop

}

private List<TaskStatusPlus> getTasks(
DruidLeaderClient druidLeaderClient,
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's name this what it is (indexing service client).

Copy link
Author

Choose a reason for hiding this comment

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

done

Copy link
Contributor

@gianm gianm left a comment

Choose a reason for hiding this comment

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

Partial review - ended around DirectDruidClient.

|size|Size of segment in bytes|
|version|Version number (generally an ISO8601 timestamp corresponding to when the segment set was first started)|
|partition_num|Partition number (an integer, unique within a datasource+interval+version; may not necessarily be contiguous)|
|num_replicas|Number replicas of this segment currently being served|
Copy link
Contributor

Choose a reason for hiding this comment

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

Number of replicas

Copy link
Author

Choose a reason for hiding this comment

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

fixed


|Column|Notes|
|------|-----|
|server|Server name in the form host:port|
Copy link
Contributor

Choose a reason for hiding this comment

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

It'd be useful to have another field for just the host. Imagine doing stuff like GROUP BY server_host to collect together all servers run on the same machine.

Copy link
Author

Choose a reason for hiding this comment

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

sure, added another column for server_host

Copy link
Contributor

Choose a reason for hiding this comment

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

Does SystemSchema support concatenating strings? Then, server isn't necessary. Users can just do select concat(host, cast(plaintext_port as VARCHAR)) from sys.servers.

Copy link
Contributor

Choose a reason for hiding this comment

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

It does support concatenating, but I think we should keep this here. It's basically the primary key of the server table, and is used for joins with the segment_servers table. The other fields (host, plaintext_port, etc) are provided too as conveniences. With system tables, since they're all generated dynamically anyway, it's ok to have some redundancy when it makes the user experience more convenient.

Copy link
Author

Choose a reason for hiding this comment

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

Just tried concat, it does not support that call cannot translate call CONCAT($t1, $t2). May be there is a way to support concat in future. But for now users can just do select server from sys.servers; :). Also server is the primary key of this table and is used in the join table segment_servers.

|------|-----|
|server|Server name in the form host:port|
|scheme|Server scheme http or https|
|server_type|Type of druid service for example historical, realtime, bridge, indexer_executor|
Copy link
Contributor

Choose a reason for hiding this comment

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

Capitalize Druid, and this sentence needs a bit more punctuation. How about:

Type of Druid service. Possible values include: historical, realtime, bridge, and indexer_executor.

Copy link
Author

Choose a reason for hiding this comment

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

changed


To retrieve information from segment_servers table, use the query:
```sql
SELECT * FROM sys.segment_servers;
Copy link
Contributor

Choose a reason for hiding this comment

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

Would be awesome to include an example here of a JOIN between "segments" and "servers". Maybe a query that shows the breakdown of number of segments for a specific datasource, by server. Something like grouping by server, filter by datasource, count segments.

Copy link
Author

Choose a reason for hiding this comment

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

Added an example of query using JOIN


### TASKS table

The tasks table provides information about active and recently-completed indexing tasks. For more information
Copy link
Contributor

Choose a reason for hiding this comment

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

"check out" not "checkout out"

Copy link
Author

Choose a reason for hiding this comment

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

fixed

{
Object[] oldRow = null;
Object[] newRow = null;
for (Object[] row : rows) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Danger! This is O(n^2): updateRow is called once per segment and must iterate over all rows (of which there is one per segment). We must use a different approach.

Copy link
Contributor

Choose a reason for hiding this comment

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

@surekhasaharan Was this comment addressed?

Copy link
Author

Choose a reason for hiding this comment

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

@jon-wei yes, the design has changed quite a lot, there is no updateRow method anymore. This was to update the num_replicas which is handled in DruidSchema now.

@@ -468,6 +468,11 @@ plan SQL queries. This metadata is cached on broker startup and also updated per
[SegmentMetadata queries](segmentmetadataquery.html). Background metadata refreshing is triggered by
segments entering and exiting the cluster, and can also be throttled through configuration.

Druid exposes system information through special system tables. There are two such schemas available: Information Schema and System Schema
Copy link
Contributor

Choose a reason for hiding this comment

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

Please use better grammar here: some punctuation is missing.

Copy link
Author

Choose a reason for hiding this comment

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

added the missing period, let me know if you want to restructure it further.

@@ -519,6 +524,89 @@ SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'druid' AND TABLE_
|COLLATION_NAME||
|JDBC_TYPE|Type code from java.sql.Types (Druid extension)|

## SYSTEM SCHEMA

The SYS schema provides visibility into Druid segments, servers and tasks.
Copy link
Contributor

Choose a reason for hiding this comment

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

It's "sys" now (not SYS).

Copy link
Author

Choose a reason for hiding this comment

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

fixed

final HttpResponseHandler<Intermediate, Final> handler
)
{
return httpClient.go(
Copy link
Contributor

Choose a reason for hiding this comment

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

No need to change this, but: IMO it would look better to write this on one line.

Copy link
Author

Choose a reason for hiding this comment

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

fixed the format

@Path("/segments")
@Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class)
public Response getDatabaseSegmentSegments()
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe just getDatabaseSegments?

Copy link
Author

Choose a reason for hiding this comment

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

yes, that looks better

@fjy
Copy link
Contributor

fjy commented Oct 6, 2018

@jihoonson any more comments?

@jihoonson
Copy link
Contributor

@fjy reviewing now.

"/simple/leader",
new ServerDiscoverySelector(EasyMock.createMock(ServiceProvider.class), "test")
)
{
Copy link
Contributor

Choose a reason for hiding this comment

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

This subclass doesn't mean empty. It just inherits all methods from the parent class.

Please check my comment below in PlannerFactory.

final @Json ObjectMapper jsonMapper
final @Json ObjectMapper jsonMapper,
final @Coordinator DruidLeaderClient coordinatorDruidLeaderClient,
final @IndexingService DruidLeaderClient overlordDruidLeaderClient
Copy link
Contributor

@jihoonson jihoonson Oct 7, 2018

Choose a reason for hiding this comment

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

Regarding my comment about mocking DruidLeaderClient in benchmark, I think the real issue is that PlannerFactory needs TimelineServerView, ObjectMapper, and DruidLeaderClients. Since this class is just a factory which is responsible for creating a new planner, it's not intuitive why it gets information from TimelineServerview or leaderClients. This makes me confused until tracking the whole change related to those parameters.

I think the same design for DruidSchema can be applied to here too. DruidSchema also needs TimelineServerView, but it's being injected and so TimelineServerView is not exposed here. Similarly, you can inject SystemSchema instead of these parameters. Actually, you've already made it injectable. Why don't you use it?

Copy link
Author

Choose a reason for hiding this comment

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

That makes sense, will pass in SystemSchema in PlannerFactory. Thanks.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks.

@@ -477,6 +477,11 @@ plan SQL queries. This metadata is cached on broker startup and also updated per
[SegmentMetadata queries](segmentmetadataquery.html). Background metadata refreshing is triggered by
segments entering and exiting the cluster, and can also be throttled through configuration.

Druid exposes system information through special system tables. There are two such schemas available: Information Schema and Sys Schema.
Information schema provides details about table and column types. The "sys" schema provides information about Druid internals like segments/tasks/servers.
Copy link
Contributor

Choose a reason for hiding this comment

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

Ah, sys is the real name. Got it.

@@ -519,6 +524,101 @@ SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'druid' AND TABLE_
|COLLATION_NAME||
|JDBC_TYPE|Type code from java.sql.Types (Druid extension)|

## SYSTEM SCHEMA

The sys schema provides visibility into Druid segments, servers and tasks.
Copy link
Contributor

Choose a reason for hiding this comment

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

Quotes look good. It makes clear that sys is the real name.

|num_rows|Number of rows in current segment, this value could be null if unkown to broker at query time|
|is_published|True if this segment has been published to the metadata store|
|is_available|True if this segment is currently being served by any server|
|is_realtime|True if this segment is being served on a realtime server|
Copy link
Contributor

Choose a reason for hiding this comment

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

We have realtime node and realtime tasks (kafka index tasks, realtime index task). I believe you meant the all types of realtime tasks. So, True if this segment is being served on any type of realtime tasks.

|num_replicas|Number of replicas of this segment currently being served|
|num_rows|Number of rows in current segment, this value could be null if unkown to broker at query time|
|is_published|True if this segment has been published to the metadata store|
|is_available|True if this segment is currently being served by any server|
Copy link
Contributor

Choose a reason for hiding this comment

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

I think server here is also ambiguous. Does it mean historicals and realtime tasks?

Copy link
Author

Choose a reason for hiding this comment

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

This means is_available is true for a segment, if it's being served by any server( either historical or realtime).

|host|Hostname of the server|
|plaintext_port|Unsecured port of the server, or -1 if plaintext traffic is disabled|
|tls_port|TLS port of the server, or -1 if TLS is disabled|
|server_type|Type of Druid service. Possible values include: historical, realtime and indexer_executor.|
Copy link
Contributor

Choose a reason for hiding this comment

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

Ok. Then probably better to add a hint like indexer_executor(peon).

});

//auth check for published segments
final CloseableIterator<DataSegment> authorizedPublishedSegments = getAuthorizedPublishedSegments(
Copy link
Contributor

Choose a reason for hiding this comment

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

This should be closed when iterating is done.

Copy link
Author

Choose a reason for hiding this comment

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

done

}
catch (JsonProcessingException e) {
log.error(e, "Error getting segment payload for segment %s", val.getKey().getIdentifier());
throw new RuntimeException(e);
Copy link
Contributor

Choose a reason for hiding this comment

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

All exceptions should be logged, so the above logging is duplicate. Also, the exception message is more important because it's propagated to the user client if necessary as well as being logged. Please remove the log and improve the exception message.

Copy link
Author

Choose a reason for hiding this comment

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

ok changed

};
}

static class BytesAccumulatingResponseHandler extends InputStreamResponseHandler
Copy link
Contributor

Choose a reason for hiding this comment

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

There are two more same classes. Please extract them to the public class and reuse it.

Copy link
Author

Choose a reason for hiding this comment

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

okay, created a common public class.

responseHandler
);
try {
future.get();
Copy link
Contributor

Choose a reason for hiding this comment

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

Ah, sorry. My question is why future needs to get here. get is called anyway in JsonParserIterator.

The query performance should be good because /druid/coordinator/v1/metadata/segments returns the StreamingOutput.

@@ -133,6 +134,14 @@ public FullResponseHolder go(Request request) throws IOException, InterruptedExc
return go(request, new FullResponseHandler(StandardCharsets.UTF_8));
}

public <Intermediate, Final> ListenableFuture<Final> goStream(
Copy link
Contributor

Choose a reason for hiding this comment

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

Please add that this method doesn't do retrying on errors or handle leader changes occurred during communication to javadoc.

return rowSignatureBuilder.build();
}

// note this is a mutable map accessed only by SystemSchema
Copy link
Contributor

Choose a reason for hiding this comment

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

Why does this need to be mutable? I don't see any places mutating this map.

Copy link
Author

Choose a reason for hiding this comment

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

this comment is obsolete now, this method returned segmentMetadataInfo before which is changing in this class.

}
}

public void setRowSignature(RowSignature rowSignature)
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you mean to remove the setters and always create a new SegmentMetadataHolder object when any of the member needs update. And replace the entry in segmentMetadataInfo map in DruidSchema every time update is required.

Yeah, this is what I meant. I think reducing the scope of concurrent access is more important here than what kind of lock types we use. As I commented here, the concurrency control algorithm is spanned over multiple places, which makes us difficult to understand and makes potential bugs even in the future.

|num_rows|Number of rows in current segment, this value could be null if unkown to broker at query time|
|is_published|True if this segment has been published to the metadata store|
|is_available|True if this segment is currently being served by any server(historical or realtime)|
|is_realtime|True if this segment is being served on any type of realtime tasks|
Copy link
Contributor

Choose a reason for hiding this comment

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

Please check the doc for these variables. Probably they should long instead of boolean?

Copy link
Author

Choose a reason for hiding this comment

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

good point, changed the doc

}
});
try {
authorizedPublishedSegments.close();
Copy link
Contributor

Choose a reason for hiding this comment

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

authorizedPublishedSegments is an iterator and the transform of publishedSegments is executed lazily. So, closing authorizedPublishedSegments here means that authorizedPublishedSegments is closed even before starting the iteration.

authorizedPublishedSegments.close() should be called when the iteration on Linq4j.asEnumerable(allSegments).where(t -> t != null) is finished. But, I'm not sure what the best way is. @gianm any idea?

Copy link
Author

Choose a reason for hiding this comment

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

Yes, i was not totally sure where to close authorizedPublishedSegments. I'll remove it from here and check what is the right place.

@surekhasaharan
Copy link
Author

Thanks so much @jihoonson for the review. I've addressed your comments, let me know if you have any further comments.

Copy link
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

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

The latest change looks good to me. @surekhasaharan thanks!

@fjy
Copy link
Contributor

fjy commented Oct 11, 2018

Spoke with @gianm about if he has any further comments and he does not so I am moving forward with merging.

@fjy fjy merged commit 3a0a667 into apache:master Oct 11, 2018
new DruidServerMetadata("server2", "server2:1234", null, 5L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
1L,
null,
ImmutableMap.of("segment2", segment2, "segment4", segment4, "segment5", segment5)
Copy link
Member

Choose a reason for hiding this comment

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

Shouldn't it use segment3

Copy link
Author

Choose a reason for hiding this comment

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

This is segment2 because in the test later, it asserts for 2 replicas for segment2. So I was trying to mimic the behavior that segment2 is served by both the servers druidServer1 and druidServer2.

Copy link
Member

Choose a reason for hiding this comment

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

It is expected that when I change the code here to "segment3", segment3, all tests in this class still pass?

Copy link
Author

Choose a reason for hiding this comment

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

Tests did pass with segment3, I checked again and the 2 replicas for segment2 are actually coming from this part , so yes it can be changed to segment3.Will do.

Copy link
Contributor

@jon-wei jon-wei Oct 13, 2018

Choose a reason for hiding this comment

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

hm, I think this suite needs a test for SystemSchema.ServerSegmentsTable that checks expected server/segment mappings, this test would catch that segment2->segment3 change

Copy link
Author

Choose a reason for hiding this comment

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

you make a great point @jon-wei , I thought server_segments is not that interesting or complex, so skipped the test. Adding the test in this PR. Since I switched segment2->segment3, that seems more cleaner. This new test will now catch segment3->segment2 change.

@surekhasaharan surekhasaharan deleted the system-table branch October 12, 2018 17:35
surekhasaharan pushed a commit to surekhasaharan/druid that referenced this pull request Oct 13, 2018
jon-wei pushed a commit that referenced this pull request Oct 13, 2018
* Add check for nullable numRows

* Make numRows long instead of Long type

* Add check for numRows in unit test
* small refactoring

* Modify test

PR comment from #6094 (review)

* Add a test for serverSegments table

* update tests
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate)
.add(segment1, index1)
.add(segment2, index2)
.add(segment2, index2)
Copy link
Member

Choose a reason for hiding this comment

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

Duplicate line

Copy link
Author

Choose a reason for hiding this comment

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

@leventov This is removed here It was because I thought about the replicas in a different way(wrong way), fixed in this new PR.

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.

8 participants