clean compile warnings

This commit is contained in:
kimchy 2010-11-26 16:03:25 +02:00
parent 6dab568a4b
commit 73e5eb9e14
14 changed files with 37 additions and 28 deletions

View File

@ -91,7 +91,7 @@ public class TransportDeleteMappingAction extends TransportMasterNodeOperationAc
final AtomicReference<Throwable> failureRef = new AtomicReference<Throwable>();
final CountDownLatch latch = new CountDownLatch(1);
deleteByQueryAction.execute(Requests.deleteByQueryRequest(request.indices()).query(QueryBuilders.filtered(QueryBuilders.matchAllQuery(), FilterBuilders.termFilter(TypeFieldMapper.NAME, request.type()))), new ActionListener<DeleteByQueryResponse>() {
deleteByQueryAction.execute(Requests.deleteByQueryRequest(request.indices()).query(QueryBuilders.filteredQuery(QueryBuilders.matchAllQuery(), FilterBuilders.termFilter(TypeFieldMapper.NAME, request.type()))), new ActionListener<DeleteByQueryResponse>() {
@Override public void onResponse(DeleteByQueryResponse deleteByQueryResponse) {
refreshAction.execute(Requests.refreshRequest(request.indices()), new ActionListener<RefreshResponse>() {
@Override public void onResponse(RefreshResponse refreshResponse) {

View File

@ -113,7 +113,7 @@ public class GeoBoundingBoxTests extends AbstractNodesTests {
client.admin().indices().prepareRefresh().execute().actionGet();
SearchResponse searchResponse = client.prepareSearch() // from NY
.setQuery(filtered(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(40.73, -74.1).bottomRight(40.717, -73.99)))
.setQuery(filteredQuery(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(40.73, -74.1).bottomRight(40.717, -73.99)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(2l));
assertThat(searchResponse.hits().hits().length, equalTo(2));
@ -178,28 +178,28 @@ public class GeoBoundingBoxTests extends AbstractNodesTests {
client.admin().indices().prepareRefresh().execute().actionGet();
SearchResponse searchResponse = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(41, -11).bottomRight(40, 9)))
.setQuery(filteredQuery(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(41, -11).bottomRight(40, 9)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(1l));
assertThat(searchResponse.hits().hits().length, equalTo(1));
assertThat(searchResponse.hits().getAt(0).id(), equalTo("2"));
searchResponse = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(41, -9).bottomRight(40, 11)))
.setQuery(filteredQuery(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(41, -9).bottomRight(40, 11)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(1l));
assertThat(searchResponse.hits().hits().length, equalTo(1));
assertThat(searchResponse.hits().getAt(0).id(), equalTo("3"));
searchResponse = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(11, 171).bottomRight(1, -169)))
.setQuery(filteredQuery(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(11, 171).bottomRight(1, -169)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(1l));
assertThat(searchResponse.hits().hits().length, equalTo(1));
assertThat(searchResponse.hits().getAt(0).id(), equalTo("5"));
searchResponse = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(9, 169).bottomRight(-1, -171)))
.setQuery(filteredQuery(matchAllQuery(), geoBoundingBoxFilter("location").topLeft(9, 169).bottomRight(-1, -171)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(1l));
assertThat(searchResponse.hits().hits().length, equalTo(1));

View File

@ -114,7 +114,7 @@ public class GeoDistanceTests extends AbstractNodesTests {
client.admin().indices().prepareRefresh().execute().actionGet();
SearchResponse searchResponse = client.prepareSearch() // from NY
.setQuery(filtered(matchAllQuery(), geoDistanceFilter("location").distance("3km").point(40.7143528, -74.0059731)))
.setQuery(filteredQuery(matchAllQuery(), geoDistanceFilter("location").distance("3km").point(40.7143528, -74.0059731)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(5l));
assertThat(searchResponse.hits().hits().length, equalTo(5));
@ -123,7 +123,7 @@ public class GeoDistanceTests extends AbstractNodesTests {
}
searchResponse = client.prepareSearch() // from NY
.setQuery(filtered(matchAllQuery(), geoDistanceFilter("location").distance("2km").point(40.7143528, -74.0059731)))
.setQuery(filteredQuery(matchAllQuery(), geoDistanceFilter("location").distance("2km").point(40.7143528, -74.0059731)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(4l));
assertThat(searchResponse.hits().hits().length, equalTo(4));
@ -132,7 +132,7 @@ public class GeoDistanceTests extends AbstractNodesTests {
}
searchResponse = client.prepareSearch() // from NY
.setQuery(filtered(matchAllQuery(), geoDistanceFilter("location").distance("1.242mi").point(40.7143528, -74.0059731)))
.setQuery(filteredQuery(matchAllQuery(), geoDistanceFilter("location").distance("1.242mi").point(40.7143528, -74.0059731)))
.execute().actionGet();
assertThat(searchResponse.hits().getTotalHits(), equalTo(4l));
assertThat(searchResponse.hits().hits().length, equalTo(4));

View File

@ -84,7 +84,7 @@ public class MatchedFiltersTests extends AbstractNodesTests {
client.admin().indices().prepareRefresh().execute().actionGet();
SearchResponse searchResponse = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), orFilter(rangeFilter("number").lte(2).filterName("test1"), rangeFilter("number").gt(2).filterName("test2"))))
.setQuery(filteredQuery(matchAllQuery(), orFilter(rangeFilter("number").lte(2).filterName("test1"), rangeFilter("number").gt(2).filterName("test2"))))
.execute().actionGet();
assertThat(searchResponse.hits().totalHits(), equalTo(3l));

View File

@ -69,7 +69,7 @@ public class SimpleQueryTests extends AbstractNodesTests {
client.admin().indices().prepareRefresh().execute().actionGet();
SearchResponse searchResponse = client.prepareSearch().setQuery(filtered(matchAllQuery(), exists("field1"))).execute().actionGet();
SearchResponse searchResponse = client.prepareSearch().setQuery(filteredQuery(matchAllQuery(), exists("field1"))).execute().actionGet();
assertThat(searchResponse.hits().totalHits(), equalTo(2l));
assertThat(searchResponse.hits().getAt(0).id(), anyOf(equalTo("1"), equalTo("2")));
assertThat(searchResponse.hits().getAt(1).id(), anyOf(equalTo("1"), equalTo("2")));
@ -84,22 +84,22 @@ public class SimpleQueryTests extends AbstractNodesTests {
assertThat(searchResponse.hits().getAt(0).id(), anyOf(equalTo("1"), equalTo("2")));
assertThat(searchResponse.hits().getAt(1).id(), anyOf(equalTo("1"), equalTo("2")));
searchResponse = client.prepareSearch().setQuery(filtered(matchAllQuery(), exists("field2"))).execute().actionGet();
searchResponse = client.prepareSearch().setQuery(filteredQuery(matchAllQuery(), exists("field2"))).execute().actionGet();
assertThat(searchResponse.hits().totalHits(), equalTo(2l));
assertThat(searchResponse.hits().getAt(0).id(), anyOf(equalTo("1"), equalTo("3")));
assertThat(searchResponse.hits().getAt(1).id(), anyOf(equalTo("1"), equalTo("3")));
searchResponse = client.prepareSearch().setQuery(filtered(matchAllQuery(), exists("field3"))).execute().actionGet();
searchResponse = client.prepareSearch().setQuery(filteredQuery(matchAllQuery(), exists("field3"))).execute().actionGet();
assertThat(searchResponse.hits().totalHits(), equalTo(1l));
assertThat(searchResponse.hits().getAt(0).id(), equalTo("4"));
searchResponse = client.prepareSearch().setQuery(filtered(matchAllQuery(), missing("field1"))).execute().actionGet();
searchResponse = client.prepareSearch().setQuery(filteredQuery(matchAllQuery(), missing("field1"))).execute().actionGet();
assertThat(searchResponse.hits().totalHits(), equalTo(2l));
assertThat(searchResponse.hits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
assertThat(searchResponse.hits().getAt(1).id(), anyOf(equalTo("3"), equalTo("4")));
// double check for cache
searchResponse = client.prepareSearch().setQuery(filtered(matchAllQuery(), missing("field1"))).execute().actionGet();
searchResponse = client.prepareSearch().setQuery(filteredQuery(matchAllQuery(), missing("field1"))).execute().actionGet();
assertThat(searchResponse.hits().totalHits(), equalTo(2l));
assertThat(searchResponse.hits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
assertThat(searchResponse.hits().getAt(1).id(), anyOf(equalTo("3"), equalTo("4")));

View File

@ -74,7 +74,7 @@ public class ScriptFilterSearchTests extends AbstractNodesTests {
logger.info("running doc['num1'].value > 1");
SearchResponse response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > 1")))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > 1")))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "doc['num1'].value")
.execute().actionGet();
@ -87,7 +87,7 @@ public class ScriptFilterSearchTests extends AbstractNodesTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").addParam("param1", 2)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").addParam("param1", 2)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "doc['num1'].value")
.execute().actionGet();
@ -98,7 +98,7 @@ public class ScriptFilterSearchTests extends AbstractNodesTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").addParam("param1", -1)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").addParam("param1", -1)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "doc['num1'].value")
.execute().actionGet();

View File

@ -62,6 +62,7 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri
return loader.parseClass(script, generateScriptName());
}
@SuppressWarnings({"unchecked"})
@Override public ExecutableScript executable(Object compiledScript, Map<String, Object> vars) {
try {
Class scriptClass = (Class) compiledScript;
@ -109,6 +110,7 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri
return script.run();
}
@SuppressWarnings({"unchecked"})
@Override public Object run(Map<String, Object> vars) {
script.getBinding().getVariables().putAll(vars);
return script.run();

View File

@ -81,7 +81,7 @@ public class GroovyScriptSearchTests {
logger.info("running doc['num1'].value > 1");
SearchResponse response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > 1").lang("groovy")))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > 1").lang("groovy")))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "groovy", "doc['num1'].value", null)
.execute().actionGet();
@ -94,7 +94,7 @@ public class GroovyScriptSearchTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("groovy").addParam("param1", 2)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("groovy").addParam("param1", 2)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "groovy", "doc['num1'].value", null)
.execute().actionGet();
@ -105,7 +105,7 @@ public class GroovyScriptSearchTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("groovy").addParam("param1", -1)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("groovy").addParam("param1", -1)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "groovy", "doc['num1'].value", null)
.execute().actionGet();
@ -119,6 +119,7 @@ public class GroovyScriptSearchTests {
assertThat((Double) response.hits().getAt(2).fields().get("sNum1").values().get(0), equalTo(3.0));
}
@SuppressWarnings({"unchecked"})
@Test public void testScriptFieldUsingSource() throws Exception {
client.admin().indices().prepareCreate("test").execute().actionGet();
client.prepareIndex("test", "type1", "1")

View File

@ -81,7 +81,7 @@ public class JavaScriptScriptSearchTests {
logger.info("running doc['num1'].value > 1");
SearchResponse response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > 1").lang("js")))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > 1").lang("js")))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "js", "doc['num1'].value", null)
.execute().actionGet();
@ -94,7 +94,7 @@ public class JavaScriptScriptSearchTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("js").addParam("param1", 2)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("js").addParam("param1", 2)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "js", "doc['num1'].value", null)
.execute().actionGet();
@ -105,7 +105,7 @@ public class JavaScriptScriptSearchTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("js").addParam("param1", -1)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("js").addParam("param1", -1)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "js", "doc['num1'].value", null)
.execute().actionGet();

View File

@ -82,7 +82,7 @@ public class PythonScriptSearchTests {
logger.info("running doc['num1'].value > 1");
SearchResponse response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > 1").lang("python")))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > 1").lang("python")))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "python", "doc['num1'].value", null)
.execute().actionGet();
@ -95,7 +95,7 @@ public class PythonScriptSearchTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("python").addParam("param1", 2)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("python").addParam("param1", 2)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "python", "doc['num1'].value", null)
.execute().actionGet();
@ -106,7 +106,7 @@ public class PythonScriptSearchTests {
logger.info("running doc['num1'].value > param1");
response = client.prepareSearch()
.setQuery(filtered(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("python").addParam("param1", -1)))
.setQuery(filteredQuery(matchAllQuery(), scriptFilter("doc['num1'].value > param1").lang("python").addParam("param1", -1)))
.addSort("num1", SortOrder.ASC)
.addScriptField("sNum1", "python", "doc['num1'].value", null)
.execute().actionGet();

View File

@ -80,6 +80,7 @@ public class CouchdbRiver extends AbstractRiverComponent implements River {
private final TransferQueue<String> stream = new LinkedTransferQueue<String>();
@SuppressWarnings({"unchecked"})
@Inject public CouchdbRiver(RiverName riverName, RiverSettings settings, @RiverIndexName String riverIndexName, Client client, ScriptService scriptService) {
super(riverName, settings);
this.riverIndexName = riverIndexName;
@ -178,6 +179,7 @@ public class CouchdbRiver extends AbstractRiverComponent implements River {
closed = true;
}
@SuppressWarnings({"unchecked"})
private String processLine(String s, BulkRequestBuilder bulk) {
Map<String, Object> ctx;
try {
@ -234,7 +236,7 @@ public class CouchdbRiver extends AbstractRiverComponent implements River {
if (closed) {
return;
}
String s = null;
String s;
try {
s = stream.take();
} catch (InterruptedException e) {
@ -295,6 +297,7 @@ public class CouchdbRiver extends AbstractRiverComponent implements River {
private class Slurper implements Runnable {
@SuppressWarnings({"unchecked"})
@Override public void run() {
while (true) {

View File

@ -68,6 +68,7 @@ public class RabbitmqRiver extends AbstractRiverComponent implements River {
private volatile ConnectionFactory connectionFactory;
@SuppressWarnings({"unchecked"})
@Inject public RabbitmqRiver(RiverName riverName, RiverSettings settings, Client client) {
super(riverName, settings);
this.client = client;

View File

@ -69,6 +69,7 @@ public class TwitterRiver extends AbstractRiverComponent implements River {
private volatile BulkRequestBuilder currentRequest;
@SuppressWarnings({"unchecked"})
@Inject public TwitterRiver(RiverName riverName, RiverSettings settings, Client client) {
super(riverName, settings);
this.client = client;

View File

@ -74,6 +74,7 @@ public class WikipediaRiver extends AbstractRiverComponent implements River {
private volatile BulkRequestBuilder currentRequest;
@SuppressWarnings({"unchecked"})
@Inject public WikipediaRiver(RiverName riverName, RiverSettings settings, Client client) throws MalformedURLException {
super(riverName, settings);
this.client = client;