Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
Forever310
druid
提交
d6f58bdc
D
druid
项目概览
Forever310
/
druid
与 Fork 源项目一致
从无法访问的项目Fork
通知
3
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
DevOps
流水线
流水线任务
计划
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
D
druid
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
DevOps
DevOps
流水线
流水线任务
计划
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
流水线任务
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
d6f58bdc
编写于
5月 29, 2014
作者:
X
Xavier Léauté
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
filtered batch inventory view tests
上级
f30d58ad
变更
1
隐藏空白更改
内联
并排
Showing
1 changed file
with
147 addition
and
10 deletion
+147
-10
server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java
.../io/druid/client/client/BatchServerInventoryViewTest.java
+147
-10
未找到文件。
server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java
浏览文件 @
d6f58bdc
...
...
@@ -20,7 +20,8 @@
package
io.druid.client.client
;
import
com.fasterxml.jackson.databind.ObjectMapper
;
import
com.google.common.base.Joiner
;
import
com.google.common.base.Predicate
;
import
com.google.common.base.Predicates
;
import
com.google.common.base.Stopwatch
;
import
com.google.common.collect.Iterables
;
import
com.google.common.collect.Sets
;
...
...
@@ -28,6 +29,7 @@ import com.google.common.util.concurrent.MoreExecutors;
import
com.metamx.common.ISE
;
import
io.druid.client.BatchServerInventoryView
;
import
io.druid.client.DruidServer
;
import
io.druid.client.ServerView
;
import
io.druid.curator.PotentiallyGzippedCompressionProvider
;
import
io.druid.curator.announcement.Announcer
;
import
io.druid.jackson.DefaultObjectMapper
;
...
...
@@ -36,17 +38,23 @@ import io.druid.server.coordination.DruidServerMetadata;
import
io.druid.server.initialization.BatchDataSegmentAnnouncerConfig
;
import
io.druid.server.initialization.ZkPathsConfig
;
import
io.druid.timeline.DataSegment
;
import
junit.framework.Assert
;
import
org.apache.curator.framework.CuratorFramework
;
import
org.apache.curator.framework.CuratorFrameworkFactory
;
import
org.apache.curator.retry.ExponentialBackoffRetry
;
import
org.apache.curator.test.TestingCluster
;
import
org.easymock.EasyMock
;
import
org.easymock.LogicalOperator
;
import
org.joda.time.DateTime
;
import
org.joda.time.Interval
;
import
org.junit.After
;
import
org.junit.Assert
;
import
org.junit.Before
;
import
org.junit.Rule
;
import
org.junit.Test
;
import
org.junit.rules.ExpectedException
;
import
javax.annotation.Nullable
;
import
java.util.Comparator
;
import
java.util.Set
;
import
java.util.concurrent.TimeUnit
;
...
...
@@ -55,6 +63,8 @@ import java.util.concurrent.TimeUnit;
public
class
BatchServerInventoryViewTest
{
private
static
final
String
testBasePath
=
"/test"
;
public
static
final
DateTime
SEGMENT_INTERVAL_START
=
new
DateTime
(
"2013-01-01"
);
public
static
final
int
INITIAL_SEGMENTS
=
100
;
private
TestingCluster
testingCluster
;
private
CuratorFramework
cf
;
...
...
@@ -63,6 +73,10 @@ public class BatchServerInventoryViewTest
private
BatchDataSegmentAnnouncer
segmentAnnouncer
;
private
Set
<
DataSegment
>
testSegments
;
private
BatchServerInventoryView
batchServerInventoryView
;
private
BatchServerInventoryView
filteredBatchServerInventoryView
;
@Rule
public
ExpectedException
exception
=
ExpectedException
.
none
();
@Before
public
void
setUp
()
throws
Exception
...
...
@@ -117,7 +131,7 @@ public class BatchServerInventoryViewTest
segmentAnnouncer
.
start
();
testSegments
=
Sets
.
newHashSet
();
for
(
int
i
=
0
;
i
<
100
;
i
++)
{
for
(
int
i
=
0
;
i
<
INITIAL_SEGMENTS
;
i
++)
{
testSegments
.
add
(
makeSegment
(
i
));
}
...
...
@@ -136,12 +150,36 @@ public class BatchServerInventoryViewTest
);
batchServerInventoryView
.
start
();
filteredBatchServerInventoryView
=
new
BatchServerInventoryView
(
new
ZkPathsConfig
()
{
@Override
public
String
getZkBasePath
()
{
return
testBasePath
;
}
},
cf
,
jsonMapper
,
new
Predicate
<
DataSegment
>()
{
@Override
public
boolean
apply
(
@Nullable
DataSegment
dataSegment
)
{
return
dataSegment
.
getInterval
().
getStart
().
isBefore
(
SEGMENT_INTERVAL_START
.
plusDays
(
INITIAL_SEGMENTS
));
}
}
);
filteredBatchServerInventoryView
.
start
();
}
@After
public
void
tearDown
()
throws
Exception
{
batchServerInventoryView
.
stop
();
filteredBatchServerInventoryView
.
stop
();
segmentAnnouncer
.
stop
();
announcer
.
stop
();
cf
.
close
();
...
...
@@ -153,7 +191,7 @@ public class BatchServerInventoryViewTest
{
segmentAnnouncer
.
announceSegments
(
testSegments
);
waitForSync
();
waitForSync
(
batchServerInventoryView
,
testSegments
);
DruidServer
server
=
Iterables
.
get
(
batchServerInventoryView
.
getInventory
(),
0
);
Set
<
DataSegment
>
segments
=
Sets
.
newHashSet
(
server
.
getSegments
().
values
());
...
...
@@ -168,7 +206,7 @@ public class BatchServerInventoryViewTest
testSegments
.
add
(
segment1
);
testSegments
.
add
(
segment2
);
waitForSync
();
waitForSync
(
batchServerInventoryView
,
testSegments
);
Assert
.
assertEquals
(
testSegments
,
Sets
.
newHashSet
(
server
.
getSegments
().
values
()));
...
...
@@ -177,32 +215,131 @@ public class BatchServerInventoryViewTest
testSegments
.
remove
(
segment1
);
testSegments
.
remove
(
segment2
);
waitForSync
();
waitForSync
(
batchServerInventoryView
,
testSegments
);
Assert
.
assertEquals
(
testSegments
,
Sets
.
newHashSet
(
server
.
getSegments
().
values
()));
}
@Test
public
void
testRunWithFilter
()
throws
Exception
{
segmentAnnouncer
.
announceSegments
(
testSegments
);
waitForSync
(
filteredBatchServerInventoryView
,
testSegments
);
DruidServer
server
=
Iterables
.
get
(
filteredBatchServerInventoryView
.
getInventory
(),
0
);
Set
<
DataSegment
>
segments
=
Sets
.
newHashSet
(
server
.
getSegments
().
values
());
Assert
.
assertEquals
(
testSegments
,
segments
);
// segment outside the range of default filter
DataSegment
segment1
=
makeSegment
(
101
);
segmentAnnouncer
.
announceSegment
(
segment1
);
testSegments
.
add
(
segment1
);
exception
.
expect
(
ISE
.
class
);
waitForSync
(
filteredBatchServerInventoryView
,
testSegments
);
}
@Test
public
void
testRunWithFilterCallback
()
throws
Exception
{
segmentAnnouncer
.
announceSegments
(
testSegments
);
waitForSync
(
filteredBatchServerInventoryView
,
testSegments
);
DruidServer
server
=
Iterables
.
get
(
filteredBatchServerInventoryView
.
getInventory
(),
0
);
Set
<
DataSegment
>
segments
=
Sets
.
newHashSet
(
server
.
getSegments
().
values
());
Assert
.
assertEquals
(
testSegments
,
segments
);
ServerView
.
SegmentCallback
callback
=
EasyMock
.
createStrictMock
(
ServerView
.
SegmentCallback
.
class
);
Comparator
<
DataSegment
>
dataSegmentComparator
=
new
Comparator
<
DataSegment
>()
{
@Override
public
int
compare
(
DataSegment
o1
,
DataSegment
o2
)
{
return
o1
.
getInterval
().
equals
(
o2
.
getInterval
())
?
0
:
-
1
;
}
};
EasyMock
.
expect
(
callback
.
segmentAdded
(
EasyMock
.<
DruidServerMetadata
>
anyObject
(),
EasyMock
.
cmp
(
makeSegment
(
INITIAL_SEGMENTS
+
2
),
dataSegmentComparator
,
LogicalOperator
.
EQUAL
)
)
)
.
andReturn
(
ServerView
.
CallbackAction
.
CONTINUE
)
.
times
(
1
);
EasyMock
.
expect
(
callback
.
segmentRemoved
(
EasyMock
.<
DruidServerMetadata
>
anyObject
(),
EasyMock
.
cmp
(
makeSegment
(
INITIAL_SEGMENTS
+
2
),
dataSegmentComparator
,
LogicalOperator
.
EQUAL
)
)
)
.
andReturn
(
ServerView
.
CallbackAction
.
CONTINUE
)
.
times
(
1
);
EasyMock
.
replay
(
callback
);
filteredBatchServerInventoryView
.
registerSegmentCallback
(
MoreExecutors
.
sameThreadExecutor
(),
callback
,
new
Predicate
<
DataSegment
>()
{
@Override
public
boolean
apply
(
@Nullable
DataSegment
dataSegment
)
{
return
dataSegment
.
getInterval
().
getStart
().
equals
(
SEGMENT_INTERVAL_START
.
plusDays
(
INITIAL_SEGMENTS
+
2
));
}
}
);
DataSegment
segment2
=
makeSegment
(
INITIAL_SEGMENTS
+
2
);
segmentAnnouncer
.
announceSegment
(
segment2
);
testSegments
.
add
(
segment2
);
DataSegment
oldSegment
=
makeSegment
(-
1
);
segmentAnnouncer
.
announceSegment
(
oldSegment
);
testSegments
.
add
(
oldSegment
);
segmentAnnouncer
.
unannounceSegment
(
oldSegment
);
testSegments
.
remove
(
oldSegment
);
waitForSync
(
filteredBatchServerInventoryView
,
testSegments
);
segmentAnnouncer
.
unannounceSegment
(
segment2
);
testSegments
.
remove
(
segment2
);
waitForSync
(
filteredBatchServerInventoryView
,
testSegments
);
EasyMock
.
verify
(
callback
);
}
private
DataSegment
makeSegment
(
int
offset
)
{
return
DataSegment
.
builder
()
.
dataSource
(
"foo"
)
.
interval
(
new
Interval
(
new
DateTime
(
"2013-01-01"
)
.
plusDays
(
offset
),
new
DateTime
(
"2013-01-02"
).
plusDays
(
offset
)
SEGMENT_INTERVAL_START
.
plusDays
(
offset
),
SEGMENT_INTERVAL_START
.
plusDays
(
offset
+
1
)
)
)
.
version
(
new
DateTime
().
toString
())
.
build
();
}
private
void
waitForSync
(
)
throws
Exception
private
static
void
waitForSync
(
BatchServerInventoryView
batchServerInventoryView
,
Set
<
DataSegment
>
testSegments
)
throws
Exception
{
Stopwatch
stopwatch
=
new
Stopwatch
().
start
();
while
(
Iterables
.
isEmpty
(
batchServerInventoryView
.
getInventory
())
||
Iterables
.
get
(
batchServerInventoryView
.
getInventory
(),
0
).
getSegments
().
size
()
!=
testSegments
.
size
())
{
Thread
.
sleep
(
500
);
if
(
stopwatch
.
elapsed
(
TimeUnit
.
MILLISECONDS
)
>
5
000
)
{
if
(
stopwatch
.
elapsed
(
TimeUnit
.
MILLISECONDS
)
>
2
000
)
{
throw
new
ISE
(
"BatchServerInventoryView is not updating"
);
}
}
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录