From 6034f96ddf69511665975984491f644bad1e61de Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Apr 2016 11:53:35 +0900 Subject: [PATCH 01/20] TAJO-2122 --- pom.xml | 1 + .../org/apache/tajo/TajoTestingCluster.java | 2 +- .../java/org/apache/tajo/conf/TajoConf.java | 1 + tajo-common/src/main/proto/tajo_protos.proto | 8 +- tajo-core-tests/pom.xml | 4 + .../tajo/querymaster/TestKillQuery.java | 3 +- ...ck.java => MockExecutionBlockContext.java} | 8 +- .../apache/tajo/worker/MockTaskExecutor.java | 2 +- .../apache/tajo/worker/MockTaskManager.java | 3 +- .../org/apache/tajo/worker/TestFetcher.java | 236 ------ .../worker/TestFetcherWithTajoPullServer.java | 440 +++++++++++ .../apache/tajo/worker/TestTaskExecutor.java | 2 +- .../tajo/querymaster/Repartitioner.java | 96 +-- .../apache/tajo/worker/AbstractFetcher.java | 84 +++ .../tajo/worker/ExecutionBlockContext.java | 10 +- .../worker/ExecutionBlockSharedResource.java | 16 + .../org/apache/tajo/worker/LocalFetcher.java | 451 ++++++++++++ .../{Fetcher.java => RemoteFetcher.java} | 159 ++-- .../org/apache/tajo/worker/TajoWorker.java | 23 +- .../java/org/apache/tajo/worker/Task.java | 2 +- .../java/org/apache/tajo/worker/TaskImpl.java | 181 ++--- .../org/apache/tajo/worker/TaskManager.java | 19 +- tajo-project/pom.xml | 12 +- tajo-pullserver/pom.xml | 6 +- .../tajo/pullserver/FileCloseListener.java | 17 +- .../tajo/pullserver/PullServerConstants.java | 93 +++ .../tajo/pullserver/PullServerUtil.java | 666 ++++++++++++++++- .../tajo/pullserver/TajoPullServer.java | 5 - .../pullserver/TajoPullServerService.java | 691 +++++------------- .../tajo/pullserver/retriever/FileChunk.java | 4 +- .../pullserver/retriever/FileChunkMeta.java | 53 ++ .../pullserver/retriever/IndexCacheKey.java | 63 ++ tajo-yarn/pom.xml | 185 +++++ .../apache/tajo/yarn/FadvisedChunkedFile.java | 82 +++ .../apache/tajo/yarn/FadvisedFileRegion.java | 171 +++++ .../apache/tajo/yarn/FileCloseListener.java | 44 ++ .../tajo/yarn/TajoPullServerService.java | 607 +++++++++++++++ 37 files changed, 3327 insertions(+), 1123 deletions(-) rename tajo-core-tests/src/test/java/org/apache/tajo/worker/{MockExecutionBlock.java => MockExecutionBlockContext.java} (79%) delete mode 100644 tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java create mode 100644 tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java rename tajo-core/src/main/java/org/apache/tajo/worker/{Fetcher.java => RemoteFetcher.java} (71%) create mode 100644 tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerConstants.java create mode 100644 tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunkMeta.java create mode 100644 tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/IndexCacheKey.java create mode 100644 tajo-yarn/pom.xml create mode 100644 tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedChunkedFile.java create mode 100644 tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java create mode 100644 tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java create mode 100644 tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java diff --git a/pom.xml b/pom.xml index d255652c8f..9a71e8e66b 100644 --- a/pom.xml +++ b/pom.xml @@ -98,6 +98,7 @@ tajo-metrics tajo-core-tests tajo-cluster-tests + tajo-yarn diff --git a/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java index 4e7d236e87..b1a33064d1 100644 --- a/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -162,7 +162,7 @@ void initPropertiesAndConfigs() { conf.setInt(ConfVars.$EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE.varname, 1); conf.setInt(ConfVars.$EXECUTOR_HASH_SHUFFLE_BUFFER_SIZE.varname, 1); - /** decrease Hbase thread and memory cache for testing */ + /* decrease Hbase thread and memory cache for testing */ //server handler conf.setInt("hbase.regionserver.handler.count", 5); //client handler diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 24a55208e3..91608a0a74 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -218,6 +218,7 @@ public static enum ConfVars implements ConfigKey { PULLSERVER_CACHE_TIMEOUT("tajo.pullserver.index-cache.timeout-min", 5, Validators.min("1")), PULLSERVER_FETCH_URL_MAX_LENGTH("tajo.pullserver.fetch-url.max-length", StorageUnit.KB, Validators.min("1")), + YARN_SHUFFLE_SERVICE_ENABLED("tajo.shuffle.yarn-service.enabled", false, Validators.bool()), SHUFFLE_SSL_ENABLED_KEY("tajo.pullserver.ssl.enabled", false, Validators.bool()), SHUFFLE_FILE_FORMAT("tajo.shuffle.file-format", BuiltinStorages.RAW, Validators.javaString()), SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", diff --git a/tajo-common/src/main/proto/tajo_protos.proto b/tajo-common/src/main/proto/tajo_protos.proto index 4d7925d876..882919a06f 100644 --- a/tajo-common/src/main/proto/tajo_protos.proto +++ b/tajo-common/src/main/proto/tajo_protos.proto @@ -49,9 +49,11 @@ enum TaskAttemptState { enum FetcherState { FETCH_INIT = 0; - FETCH_FETCHING = 1; - FETCH_FINISHED = 2; - FETCH_FAILED = 3; + FETCH_META_FETCHING = 1; + FETCH_META_FINISHED = 2; + FETCH_DATA_FETCHING = 3; + FETCH_DATA_FINISHED = 4; + FETCH_FAILED = 5; } message WorkerConnectionInfoProto { diff --git a/tajo-core-tests/pom.xml b/tajo-core-tests/pom.xml index b12642a50d..3554df4cae 100644 --- a/tajo-core-tests/pom.xml +++ b/tajo-core-tests/pom.xml @@ -172,6 +172,10 @@ org.apache.tajo tajo-pullserver + + org.apache.tajo + tajo-yarn + org.apache.tajo tajo-rpc-protobuf diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index ac5efd9121..8d33dbc986 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -26,7 +26,6 @@ import org.apache.tajo.ResourceProtos.ExecutionBlockContextResponse; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.CatalogService; -import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.client.TajoClient; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.parser.sql.SQLAnalyzer; @@ -269,7 +268,7 @@ public NodeResourceManager getNodeResourceManager() { } }; - ExecutionBlockContext context = new MockExecutionBlock(workerContext, requestProtoBuilder.build()) { + ExecutionBlockContext context = new MockExecutionBlockContext(workerContext, requestProtoBuilder.build()) { @Override public Path createBaseDir() throws IOException { return new Path("test"); diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockExecutionBlockContext.java similarity index 79% rename from tajo-core-tests/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java rename to tajo-core-tests/src/test/java/org/apache/tajo/worker/MockExecutionBlockContext.java index cbc4312de9..b64ab9bbf6 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockExecutionBlockContext.java @@ -23,11 +23,11 @@ import java.io.IOException; -public class MockExecutionBlock extends ExecutionBlockContext { +public class MockExecutionBlockContext extends ExecutionBlockContext { - public MockExecutionBlock(TajoWorker.WorkerContext workerContext, - ExecutionBlockContextResponse request) throws IOException { - super(workerContext, request, null); + public MockExecutionBlockContext(TajoWorker.WorkerContext workerContext, + ExecutionBlockContextResponse request) throws IOException { + super(workerContext, request, null, null); } @Override diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java index 071d26ac2d..ea609b14bb 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -145,7 +145,7 @@ public TaskHistory createTaskHistory() { } @Override - public List getFetchers() { + public List getFetchers() { return null; } }; diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskManager.java index 5979bbbaa2..0e114bb443 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskManager.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/MockTaskManager.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeoutException; public class MockTaskManager extends TaskManager { @@ -61,7 +60,7 @@ protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionB .setQueryContext(new QueryContext(new TajoConf()).getProto()) .setQueryOutputPath("testpath") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); - return new MockExecutionBlock(getWorkerContext(), builder.build()); + return new MockExecutionBlockContext(getWorkerContext(), builder.build()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java deleted file mode 100644 index dfc37b0769..0000000000 --- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ /dev/null @@ -1,236 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.worker; - -import org.apache.hadoop.fs.*; -import org.apache.tajo.QueryId; -import org.apache.tajo.QueryIdFactory; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.pullserver.TajoPullServerService; -import org.apache.tajo.pullserver.retriever.FileChunk; -import org.apache.tajo.storage.HashShuffleAppenderManager; -import org.apache.tajo.util.CommonTestingUtil; -import org.junit.*; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.Random; - -import static org.junit.Assert.*; - -public class TestFetcher { - private String TEST_DATA = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFetcher"; - private String INPUT_DIR = TEST_DATA+"/in/"; - private String OUTPUT_DIR = TEST_DATA+"/out/"; - private TajoConf conf = new TajoConf(); - private TajoPullServerService pullServerService; - - @Before - public void setUp() throws Exception { - CommonTestingUtil.getTestDir(TEST_DATA); - CommonTestingUtil.getTestDir(INPUT_DIR); - CommonTestingUtil.getTestDir(OUTPUT_DIR); - conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, INPUT_DIR); - conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_TIMEOUT, 1); - conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE, 127); - - pullServerService = new TajoPullServerService(); - pullServerService.init(conf); - pullServerService.start(); - } - - @After - public void tearDown(){ - pullServerService.stop(); - } - - @Test - public void testGet() throws IOException { - Random rnd = new Random(); - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; - String sid = "1"; - String partId = "1"; - - int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); - String dataPath = conf.getVar(ConfVars.WORKER_TEMPORAL_DIR) + - queryId.toString() + "/output/" + sid + "/hash-shuffle/" + partParentId + "/" + partId; - - String params = String.format("qid=%s&sid=%s&p=%s&type=%s", queryId, sid, partId, "h"); - - Path inputPath = new Path(dataPath); - FSDataOutputStream stream = FileSystem.getLocal(conf).create(inputPath, true); - for (int i = 0; i < 100; i++) { - String data = ""+rnd.nextInt(); - stream.write(data.getBytes()); - } - stream.flush(); - stream.close(); - - URI uri = URI.create("http://127.0.0.1:" + pullServerService.getPort() + "/?" + params); - FileChunk storeChunk = new FileChunk(new File(OUTPUT_DIR + "data"), 0, 0); - storeChunk.setFromRemote(true); - final Fetcher fetcher = new Fetcher(conf, uri, storeChunk); - FileChunk chunk = fetcher.get().get(0); - assertNotNull(chunk); - assertNotNull(chunk.getFile()); - - FileSystem fs = FileSystem.getLocal(new TajoConf()); - FileStatus inStatus = fs.getFileStatus(inputPath); - FileStatus outStatus = fs.getFileStatus(new Path(OUTPUT_DIR, "data")); - - assertEquals(inStatus.getLen(), outStatus.getLen()); - assertEquals(TajoProtos.FetcherState.FETCH_FINISHED, fetcher.getState()); - } - - @Test - public void testAdjustFetchProcess() { - assertEquals(0.0f, TaskImpl.adjustFetchProcess(0, 0), 0); - assertEquals(0.0f, TaskImpl.adjustFetchProcess(10, 10), 0); - assertEquals(0.05f, TaskImpl.adjustFetchProcess(10, 9), 0); - assertEquals(0.1f, TaskImpl.adjustFetchProcess(10, 8), 0); - assertEquals(0.25f, TaskImpl.adjustFetchProcess(10, 5), 0); - assertEquals(0.45f, TaskImpl.adjustFetchProcess(10, 1), 0); - assertEquals(0.5f, TaskImpl.adjustFetchProcess(10, 0), 0); - } - - @Test - public void testStatus() throws Exception { - Random rnd = new Random(); - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; - String sid = "1"; - String ta = "1_0"; - String partId = "1"; - - String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + sid + "/" +ta + "/output/" + partId; - String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", queryId, sid, partId, "h", ta); - - FSDataOutputStream stream = FileSystem.getLocal(conf).create(new Path(dataPath), true); - for (int i = 0; i < 100; i++) { - String data = ""+rnd.nextInt(); - stream.write(data.getBytes()); - } - stream.flush(); - stream.close(); - - URI uri = URI.create("http://127.0.0.1:" + pullServerService.getPort() + "/?" + params); - FileChunk storeChunk = new FileChunk(new File(OUTPUT_DIR + "data"), 0, 0); - storeChunk.setFromRemote(true); - final Fetcher fetcher = new Fetcher(conf, uri, storeChunk); - assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); - - fetcher.get(); - assertEquals(TajoProtos.FetcherState.FETCH_FINISHED, fetcher.getState()); - } - - @Test - public void testNoContentFetch() throws Exception { - - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; - String sid = "1"; - String ta = "1_0"; - String partId = "1"; - - String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + sid + "/" +ta + "/output/" + partId; - String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", queryId, sid, partId, "h", ta); - - Path inputPath = new Path(dataPath); - FileSystem fs = FileSystem.getLocal(conf); - if(fs.exists(inputPath)){ - fs.delete(new Path(dataPath), true); - } - - FSDataOutputStream stream = FileSystem.getLocal(conf).create(new Path(dataPath).getParent(), true); - stream.close(); - - URI uri = URI.create("http://127.0.0.1:" + pullServerService.getPort() + "/?" + params); - FileChunk storeChunk = new FileChunk(new File(OUTPUT_DIR + "data"), 0, 0); - storeChunk.setFromRemote(true); - final Fetcher fetcher = new Fetcher(conf, uri, storeChunk); - assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); - - fetcher.get(); - assertEquals(TajoProtos.FetcherState.FETCH_FINISHED, fetcher.getState()); - } - - @Test - public void testFailureStatus() throws Exception { - Random rnd = new Random(); - - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; - String sid = "1"; - String ta = "1_0"; - String partId = "1"; - - String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + sid + "/" +ta + "/output/" + partId; - - //TajoPullServerService will be throws BAD_REQUEST by Unknown shuffle type - String shuffleType = "x"; - String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", queryId, sid, partId, shuffleType, ta); - - FSDataOutputStream stream = FileSystem.getLocal(conf).create(new Path(dataPath), true); - - for (int i = 0; i < 100; i++) { - String data = params + rnd.nextInt(); - stream.write(data.getBytes()); - } - stream.flush(); - stream.close(); - - URI uri = URI.create("http://127.0.0.1:" + pullServerService.getPort() + "/?" + params); - FileChunk storeChunk = new FileChunk(new File(OUTPUT_DIR + "data"), 0, 0); - storeChunk.setFromRemote(true); - final Fetcher fetcher = new Fetcher(conf, uri, storeChunk); - assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); - - fetcher.get(); - assertEquals(TajoProtos.FetcherState.FETCH_FAILED, fetcher.getState()); - } - - @Test - public void testServerFailure() throws Exception { - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; - String sid = "1"; - String ta = "1_0"; - String partId = "1"; - - String dataPath = INPUT_DIR + queryId.toString() + "/output"+ "/" + sid + "/" +ta + "/output/" + partId; - String params = String.format("qid=%s&sid=%s&p=%s&type=%s&ta=%s", queryId, sid, partId, "h", ta); - - URI uri = URI.create("http://127.0.0.1:" + pullServerService.getPort() + "/?" + params); - FileChunk storeChunk = new FileChunk(new File(OUTPUT_DIR + "data"), 0, 0); - storeChunk.setFromRemote(true); - final Fetcher fetcher = new Fetcher(conf, uri, storeChunk); - assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); - - pullServerService.stop(); - - boolean failure = false; - try{ - fetcher.get(); - } catch (Throwable e){ - failure = true; - } - assertTrue(failure); - assertEquals(TajoProtos.FetcherState.FETCH_FAILED, fetcher.getState()); - } -} diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java new file mode 100644 index 0000000000..9c364afbbc --- /dev/null +++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java @@ -0,0 +1,440 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import com.google.common.collect.Lists; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.service.Service; +import org.apache.tajo.QueryId; +import org.apache.tajo.QueryIdFactory; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.FetcherState; +import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.SchemaFactory; +import org.apache.tajo.catalog.SortSpec; +import org.apache.tajo.common.TajoDataTypes.Type; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.pullserver.PullServerConstants; +import org.apache.tajo.pullserver.PullServerUtil; +import org.apache.tajo.pullserver.PullServerUtil.PullServerRequestURIBuilder; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.index.bst.BSTIndex; +import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexWriter; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.worker.FetchImpl.RangeParam; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.*; + +import static org.junit.Assert.*; + +@RunWith(Parameterized.class) +public class TestFetcherWithTajoPullServer { + private enum FetchType { + LOCAL, + REMOTE + } + private enum PullServerType { + TAJO, + YARN + } + + private final String TEST_DATA = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/" + + TestFetcherWithTajoPullServer.class.getSimpleName(); + private final String INPUT_DIR = TEST_DATA+"/in/"; + private final String OUTPUT_DIR = TEST_DATA+"/out/"; + private final TajoConf conf = new TajoConf(); + private Service pullServerService; + private final int maxUrlLength = conf.getIntVar(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH); + private final String TEST_TABLE_NAME = "test"; + private final FetchType fetchType; + private final PullServerType pullServerType; + private int pullserverPort; + + public TestFetcherWithTajoPullServer(FetchType fetchType, PullServerType pullServerType) { + this.fetchType = fetchType; + this.pullServerType = pullServerType; + } + + @Before + public void setUp() throws Exception { + CommonTestingUtil.getTestDir(TEST_DATA); + CommonTestingUtil.getTestDir(INPUT_DIR); + CommonTestingUtil.getTestDir(OUTPUT_DIR); + conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, INPUT_DIR); + conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_TIMEOUT, 1); + conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE, 127); + + if (pullServerType.equals(PullServerType.TAJO)) { + pullServerService = new TajoPullServerService(); + } else { + pullServerService = new org.apache.tajo.yarn.TajoPullServerService(); + } + pullServerService.init(conf); + pullServerService.start(); + + if (pullServerType.equals(PullServerType.TAJO)) { + pullserverPort = ((TajoPullServerService)pullServerService).getPort(); + } else { + pullserverPort = ((org.apache.tajo.yarn.TajoPullServerService)pullServerService).getPort(); + } + } + + @After + public void tearDown() { + pullServerService.stop(); + } + + @Parameters(name = "{index}: {0}, {1}") + public static Collection generateParameters() { + return Arrays.asList(new Object[][] { + {FetchType.LOCAL, PullServerType.TAJO}, + {FetchType.REMOTE, PullServerType.TAJO}, + {FetchType.LOCAL, PullServerType.YARN}, + {FetchType.REMOTE, PullServerType.YARN} + }); + } + + private AbstractFetcher getFetcher(URI uri, File data) throws IOException { + if (fetchType.equals(FetchType.LOCAL)) { + return new LocalFetcher(conf, uri, TEST_TABLE_NAME); + } else { + FileChunk storeChunk = new FileChunk(data, 0, data.length()); + storeChunk.setFromRemote(true); + return new RemoteFetcher(conf, uri, storeChunk); + } + } + + @Test + public void testGetHashShuffle() throws IOException { + Random rnd = new Random(); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + String sid = "1"; + String partId = "1"; + + Path queryBaseDir = PullServerUtil.getBaseOutputDir(queryId.toString(), sid); + final int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); + final Path dataPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); + + PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder("127.0.0.1", pullserverPort, + maxUrlLength); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(queryId.toString()) + .setEbId(sid) + .setPartId(partId) + .setShuffleType(PullServerConstants.HASH_SHUFFLE_PARAM_STRING); + + Path inputPath = new Path(INPUT_DIR, dataPath); + FSDataOutputStream stream = FileSystem.getLocal(conf).create(inputPath, true); + for (int i = 0; i < 100; i++) { + String data = ""+rnd.nextInt(); + stream.write(data.getBytes()); + } + stream.flush(); + stream.close(); + + URI uri = builder.build(false).get(0); + File data = new File(OUTPUT_DIR + "data"); + + final AbstractFetcher fetcher = getFetcher(uri, data); + + FileChunk chunk = fetcher.get().get(0); + assertNotNull(chunk); + assertNotNull(chunk.getFile()); + + FileSystem fs = FileSystem.getLocal(new TajoConf()); + FileStatus inStatus = fs.getFileStatus(inputPath); + FileStatus outStatus = fs.getFileStatus(new Path(chunk.getFile().getAbsolutePath())); + + assertEquals(inStatus.getLen(), outStatus.getLen()); + assertEquals(FetcherState.FETCH_DATA_FINISHED, fetcher.getState()); + } + + @Test + public void testGetRangeShuffle() throws IOException { + Random rnd = new Random(); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + String sid = "1"; + String partId = "1"; + String taskId = "1"; + String attemptId = "0"; + + Path queryBaseDir = PullServerUtil.getBaseOutputDir(queryId.toString(), sid); + Path outDir = StorageUtil.concatPath(queryBaseDir, taskId + "_" + attemptId, "output"); + Path dataPath = StorageUtil.concatPath(outDir, "output"); + Path indexPath = StorageUtil.concatPath(outDir, "index"); + + List strings = new ArrayList<>(100); + for (int i = 0; i < 100; i++) { + strings.add("" + rnd.nextInt()); + } + Collections.sort(strings); + + Path inputPath = new Path(INPUT_DIR, dataPath); + FileSystem fs = FileSystem.getLocal(conf); + if (fs.exists(outDir)) { + fs.delete(outDir, true); + } + final FSDataOutputStream stream = fs.create(inputPath, true); + BSTIndex index = new BSTIndex(conf); + Schema schema = SchemaFactory.newV1(new Column[] {new Column("rnd", Type.TEXT)}); + SortSpec[] sortSpecs = new SortSpec[] {new SortSpec(schema.getColumn(0))}; + BSTIndexWriter writer = index.getIndexWriter(new Path(INPUT_DIR, indexPath), BSTIndex.TWO_LEVEL_INDEX, schema, new BaseTupleComparator(schema, sortSpecs), true); + writer.init(); + + for (String t : strings) { + writer.write(new VTuple(new Datum[] {DatumFactory.createText(t)}), stream.getPos()); + stream.write(t.getBytes()); + } + stream.flush(); + writer.flush(); + stream.close(); + writer.close(); + + RangeParam rangeParam = new RangeParam(new TupleRange(sortSpecs, + new VTuple(new Datum[] {DatumFactory.createText(strings.get(0))}), + new VTuple(new Datum[] {DatumFactory.createText(strings.get(strings.size() - 1))})), true, RowStoreUtil.createEncoder(schema)); + PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder("127.0.0.1", pullserverPort, + maxUrlLength); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(queryId.toString()) + .setEbId(sid) + .setPartId(partId) + .setShuffleType(PullServerConstants.RANGE_SHUFFLE_PARAM_STRING) + .setTaskIds(Lists.newArrayList(Integer.parseInt(taskId))) + .setAttemptIds(Lists.newArrayList(Integer.parseInt(attemptId))) + .setStartKeyBase64(new String(Base64.encodeBase64(rangeParam.getStart()))) + .setEndKeyBase64(new String(Base64.encodeBase64(rangeParam.getEnd()))) + .setLastInclude(true); + + URI uri = builder.build(true).get(0); + File data = new File(OUTPUT_DIR + "data"); + + final AbstractFetcher fetcher = getFetcher(uri, data); + + FileChunk chunk = fetcher.get().get(0); + assertNotNull(chunk); + assertNotNull(chunk.getFile()); + + FileStatus inStatus = fs.getFileStatus(inputPath); + FileStatus outStatus = fs.getFileStatus(new Path(chunk.getFile().getAbsolutePath())); + + assertEquals(inStatus.getLen(), outStatus.getLen()); + assertEquals(FetcherState.FETCH_DATA_FINISHED, fetcher.getState()); + } + + @Test + public void testAdjustFetchProcess() { + Assert.assertEquals(0.0f, TaskImpl.adjustFetchProcess(0, 0), 0); + assertEquals(0.0f, TaskImpl.adjustFetchProcess(10, 10), 0); + assertEquals(0.05f, TaskImpl.adjustFetchProcess(10, 9), 0); + assertEquals(0.1f, TaskImpl.adjustFetchProcess(10, 8), 0); + assertEquals(0.25f, TaskImpl.adjustFetchProcess(10, 5), 0); + assertEquals(0.45f, TaskImpl.adjustFetchProcess(10, 1), 0); + assertEquals(0.5f, TaskImpl.adjustFetchProcess(10, 0), 0); + } + + @Test + public void testStatus() throws Exception { + Random rnd = new Random(); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + String sid = "1"; + String ta = "1_0"; + String partId = "1"; + + Path queryBaseDir = PullServerUtil.getBaseOutputDir(queryId.toString(), sid); + final int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); + final Path dataPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); + + PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder("127.0.0.1", pullserverPort, + maxUrlLength); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(queryId.toString()) + .setEbId(sid) + .setPartId(partId) + .setShuffleType(PullServerConstants.HASH_SHUFFLE_PARAM_STRING) + .setTaskAttemptIds(Lists.newArrayList(ta)); + + FSDataOutputStream stream = FileSystem.getLocal(conf).create(new Path(INPUT_DIR, dataPath), true); + for (int i = 0; i < 100; i++) { + String data = ""+rnd.nextInt(); + stream.write(data.getBytes()); + } + stream.flush(); + stream.close(); + + URI uri = builder.build(true).get(0); + File data = new File(OUTPUT_DIR + "data"); + final AbstractFetcher fetcher = getFetcher(uri, data); + assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); + + fetcher.get(); + assertEquals(FetcherState.FETCH_DATA_FINISHED, fetcher.getState()); + } + + @Test + public void testNoContentFetch() throws Exception { + + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + String sid = "1"; + String ta = "1_0"; + String partId = "1"; + + Path queryBaseDir = PullServerUtil.getBaseOutputDir(queryId.toString(), sid); + final int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); + final Path dataPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); + + PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder("127.0.0.1", pullserverPort, + maxUrlLength); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(queryId.toString()) + .setEbId(sid) + .setPartId(partId) + .setShuffleType(PullServerConstants.HASH_SHUFFLE_PARAM_STRING) + .setTaskAttemptIds(Lists.newArrayList(ta)); + + Path inputPath = new Path(INPUT_DIR, dataPath); + FileSystem fs = FileSystem.getLocal(conf); + if(fs.exists(inputPath)){ + fs.delete(inputPath, true); + } + + FSDataOutputStream stream = fs.create(inputPath, true); + stream.close(); + + URI uri = builder.build(true).get(0); + File data = new File(OUTPUT_DIR + "data"); + final AbstractFetcher fetcher = getFetcher(uri, data); + assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); + + try { + fetcher.get(); + if (fetchType.equals(FetchType.LOCAL)) { + fail(); + } + } catch (IOException e) { + if (fetchType.equals(FetchType.REMOTE)) { + fail(); + } + } + assertEquals(FetcherState.FETCH_FAILED, fetcher.getState()); + } + + @Test + public void testFailureStatus() throws Exception { + Random rnd = new Random(); + + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + String sid = "1"; + String ta = "1_0"; + String partId = "1"; + + Path queryBaseDir = PullServerUtil.getBaseOutputDir(queryId.toString(), sid); + final int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); + final Path dataPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); + + PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder("127.0.0.1", pullserverPort, + maxUrlLength); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(queryId.toString()) + .setEbId(sid) + .setPartId(partId) + .setShuffleType("x") //TajoPullServerService will be throws BAD_REQUEST by Unknown shuffle type + .setTaskAttemptIds(Lists.newArrayList(ta)); + + FSDataOutputStream stream = FileSystem.getLocal(conf).create(new Path(INPUT_DIR, dataPath), true); + + for (int i = 0; i < 100; i++) { + String data = "" + rnd.nextInt(); + stream.write(data.getBytes()); + } + stream.flush(); + stream.close(); + + URI uri = builder.build(true).get(0); + File data = new File(OUTPUT_DIR + "data"); + final AbstractFetcher fetcher = getFetcher(uri, data); + assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); + + try { + fetcher.get(); + if (fetchType.equals(FetchType.LOCAL)) { + fail(); + } + } catch (IllegalArgumentException e) { + if (!fetchType.equals(FetchType.LOCAL)) { + fail(); + } + } + assertEquals(TajoProtos.FetcherState.FETCH_FAILED, fetcher.getState()); + } + + @Test + public void testServerFailure() throws Exception { + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + String sid = "1"; + String ta = "1_0"; + String partId = "1"; + + PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder("127.0.0.1", pullserverPort, + maxUrlLength); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(queryId.toString()) + .setEbId(sid) + .setPartId(partId) + .setShuffleType(PullServerConstants.HASH_SHUFFLE_PARAM_STRING) + .setTaskAttemptIds(Lists.newArrayList(ta)); + + URI uri = builder.build(true).get(0); + File data = new File(OUTPUT_DIR + "data"); + final AbstractFetcher fetcher = getFetcher(uri, data); + assertEquals(TajoProtos.FetcherState.FETCH_INIT, fetcher.getState()); + + pullServerService.stop(); + + boolean failure = false; + try{ + fetcher.get(); + } catch (IOException e){ + failure = true; + } + assertTrue(failure); + assertEquals(TajoProtos.FetcherState.FETCH_FAILED, fetcher.getState()); + } +} diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index 45e430e9e2..df5b3c8ac9 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -305,7 +305,7 @@ public TaskHistory createTaskHistory() { } @Override - public List getFetchers() { + public List getFetchers() { return null; } }; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java index 8fdd6ce38e..d725c73509 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java @@ -50,6 +50,8 @@ import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage; import org.apache.tajo.plan.serder.PlanProto.EnforceProperty; import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.pullserver.PullServerConstants; +import org.apache.tajo.pullserver.PullServerUtil.PullServerRequestURIBuilder; import org.apache.tajo.querymaster.Task.IntermediateEntry; import org.apache.tajo.querymaster.Task.PullHost; import org.apache.tajo.storage.*; @@ -70,7 +72,6 @@ import java.util.*; import java.util.Map.Entry; import java.util.stream.Collectors; -import java.util.stream.IntStream; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.*; @@ -1124,89 +1125,32 @@ private static String getRangeParam(FetchProto proto) { } public static List createFetchURL(int maxUrlLength, FetchProto fetch, boolean includeParts) { - String scheme = "http://"; - - StringBuilder urlPrefix = new StringBuilder(scheme); + PullServerRequestURIBuilder builder = + new PullServerRequestURIBuilder(fetch.getHost(), fetch.getPort(), maxUrlLength); ExecutionBlockId ebId = new ExecutionBlockId(fetch.getExecutionBlockId()); - urlPrefix.append(fetch.getHost()).append(":").append(fetch.getPort()).append("/?") - .append("qid=").append(ebId.getQueryId().toString()) - .append("&sid=").append(ebId.getId()) - .append("&p=").append(fetch.getPartitionId()) - .append("&type="); + builder.setRequestType(PullServerConstants.CHUNK_REQUEST_PARAM_STRING) + .setQueryId(ebId.getQueryId().toString()) + .setEbId(ebId.getId()) + .setPartId(fetch.getPartitionId()); + if (fetch.getType() == HASH_SHUFFLE) { - urlPrefix.append("h"); + builder.setShuffleType(PullServerConstants.HASH_SHUFFLE_PARAM_STRING); } else if (fetch.getType() == RANGE_SHUFFLE) { - urlPrefix.append("r").append("&").append(getRangeParam(fetch)); + builder.setShuffleType(PullServerConstants.RANGE_SHUFFLE_PARAM_STRING); + builder.setStartKeyBase64(new String(org.apache.commons.codec.binary.Base64.encodeBase64(fetch.getRangeStart().toByteArray()))); + builder.setEndKeyBase64(new String(org.apache.commons.codec.binary.Base64.encodeBase64(fetch.getRangeEnd().toByteArray()))); + builder.setLastInclude(fetch.getRangeLastInclusive()); } else if (fetch.getType() == SCATTERED_HASH_SHUFFLE) { - urlPrefix.append("s"); + builder.setShuffleType(PullServerConstants.SCATTERED_HASH_SHUFFLE_PARAM_STRING); } - if (fetch.getLength() >= 0) { - urlPrefix.append("&offset=").append(fetch.getOffset()).append("&length=").append(fetch.getLength()); + builder.setOffset(fetch.getOffset()).setLength(fetch.getLength()); } - - List fetchURLs = new ArrayList<>(); - if(includeParts) { - if (fetch.getType() == HASH_SHUFFLE || fetch.getType() == SCATTERED_HASH_SHUFFLE) { - fetchURLs.add(URI.create(urlPrefix.toString())); - } else { - urlPrefix.append("&ta="); - // If the get request is longer than 2000 characters, - // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long. - // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15 - // The below code transforms a long request to multiple requests. - List taskIdsParams = new ArrayList<>(); - StringBuilder taskIdListBuilder = new StringBuilder(); - - final List taskIds = fetch.getTaskIdList(); - final List attemptIds = fetch.getAttemptIdList(); - - // Sort task ids to increase cache hit in pull server - final List> taskAndAttemptIds = IntStream.range(0, taskIds.size()) - .mapToObj(i -> new Pair<>(taskIds.get(i), attemptIds.get(i))) - .sorted((p1, p2) -> p1.getFirst() - p2.getFirst()) - .collect(Collectors.toList()); - - boolean first = true; - - for (int i = 0; i < taskAndAttemptIds.size(); i++) { - StringBuilder taskAttemptId = new StringBuilder(); - - if (!first) { // when comma is added? - taskAttemptId.append(","); - } else { - first = false; - } - - int taskId = taskAndAttemptIds.get(i).getFirst(); - if (taskId < 0) { - // In the case of hash shuffle each partition has single shuffle file per worker. - // TODO If file is large, consider multiple fetching(shuffle file can be split) - continue; - } - int attemptId = taskAndAttemptIds.get(i).getSecond(); - taskAttemptId.append(taskId).append("_").append(attemptId); - - if (urlPrefix.length() + taskIdListBuilder.length() > maxUrlLength) { - taskIdsParams.add(taskIdListBuilder.toString()); - taskIdListBuilder = new StringBuilder(taskId + "_" + attemptId); - } else { - taskIdListBuilder.append(taskAttemptId); - } - } - // if the url params remain - if (taskIdListBuilder.length() > 0) { - taskIdsParams.add(taskIdListBuilder.toString()); - } - for (String param : taskIdsParams) { - fetchURLs.add(URI.create(urlPrefix + param)); - } - } - } else { - fetchURLs.add(URI.create(urlPrefix.toString())); + if (includeParts) { + builder.setTaskIds(fetch.getTaskIdList()); + builder.setAttemptIds(fetch.getAttemptIdList()); } - - return fetchURLs; + return builder.build(includeParts); } public static Map> hashByKey(List entries) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java new file mode 100644 index 0000000000..b36c5fc8ad --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.FetcherState; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.pullserver.retriever.FileChunk; + +import java.io.IOException; +import java.net.URI; +import java.util.List; + +public abstract class AbstractFetcher { + + protected final URI uri; + protected FileChunk fileChunk; + protected final TajoConf conf; + + protected TajoProtos.FetcherState state; + + protected long startTime; + protected volatile long finishTime; + protected long fileLen; + protected int messageReceiveCount; + + public AbstractFetcher(TajoConf conf, URI uri) { + this(conf, uri, null); + } + + public AbstractFetcher(TajoConf conf, URI uri, FileChunk fileChunk) { + this.conf = conf; + this.uri = uri; + this.fileChunk = fileChunk; + this.state = TajoProtos.FetcherState.FETCH_INIT; + } + + public URI getURI() { + return this.uri; + } + + public long getStartTime() { + return startTime; + } + + public long getFinishTime() { + return finishTime; + } + + public long getFileLen() { + return fileLen; + } + + public TajoProtos.FetcherState getState() { + return state; + } + + public int getMessageReceiveCount() { + return messageReceiveCount; + } + + public abstract List get() throws IOException; + + protected void endFetch(FetcherState state) { + this.finishTime = System.currentTimeMillis(); + this.state = state; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index e675d7063d..4ab6627e8c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -37,6 +37,7 @@ import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.exception.ErrorUtil; @@ -44,6 +45,7 @@ import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.pullserver.PullServerUtil; import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.rpc.*; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; @@ -103,7 +105,7 @@ public class ExecutionBlockContext { private final Map taskHistories = Maps.newConcurrentMap(); public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, ExecutionBlockContextResponse request, - AsyncRpcClient queryMasterClient) + AsyncRpcClient queryMasterClient, @Nullable TajoPullServerService pullServerService) throws IOException { this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); this.connManager = RpcClientManager.getInstance(); @@ -117,7 +119,7 @@ public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, ExecutionBl this.queryEngine = new TajoQueryEngine(systemConf); this.queryContext = new QueryContext(workerContext.getConf(), request.getQueryContext()); this.plan = request.getPlanJson(); - this.resource = new ExecutionBlockSharedResource(); + this.resource = new ExecutionBlockSharedResource(pullServerService); this.workerContext = workerContext; this.shuffleType = request.getShuffleType(); this.queryMasterClient = queryMasterClient; @@ -281,12 +283,12 @@ public Path createBaseDir() throws IOException { } public static Path getBaseOutputDir(ExecutionBlockId executionBlockId) { - return TajoPullServerService.getBaseOutputDir( + return PullServerUtil.getBaseOutputDir( executionBlockId.getQueryId().toString(), String.valueOf(executionBlockId.getId())); } public static Path getBaseInputDir(ExecutionBlockId executionBlockId) { - return TajoPullServerService.getBaseInputDir(executionBlockId.getQueryId().toString(), executionBlockId.toString()); + return PullServerUtil.getBaseInputDir(executionBlockId.getQueryId().toString(), executionBlockId.toString()); } public ExecutionBlockId getExecutionBlockId() { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java index d55c4bacf0..7e44bc35c6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.SessionVars; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Schema; import org.apache.tajo.engine.codegen.ExecutorPreCompiler; import org.apache.tajo.engine.codegen.TajoClassLoader; @@ -34,8 +35,10 @@ import org.apache.tajo.exception.TajoException; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.logical.LogicalNode; +import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.util.Pair; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; public class ExecutionBlockSharedResource { @@ -52,6 +55,15 @@ public class ExecutionBlockSharedResource { private ExecutorPreCompiler.CompilationContext compilationContext; private LogicalNode plan; private boolean codeGenEnabled = false; + private final TajoPullServerService pullServerService; + + public ExecutionBlockSharedResource() { + this(null); + } + + public ExecutionBlockSharedResource(@Nullable TajoPullServerService pullServerService) { + this.pullServerService = pullServerService; + } public void initialize(final QueryContext context, final String planJson) { @@ -134,6 +146,10 @@ public void releaseBroadcastCache(ExecutionBlockId id) { TableCache.getInstance().releaseCache(id); } + public Optional getPullServerService() { + return pullServerService == null ? Optional.empty() : Optional.of(pullServerService); + } + public void release() { compilationContext = null; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java new file mode 100644 index 0000000000..85f397f16e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java @@ -0,0 +1,451 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + package org.apache.tajo.worker; + +import com.google.common.annotations.VisibleForTesting; +import com.google.gson.Gson; +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.channel.*; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.http.*; +import io.netty.handler.timeout.ReadTimeoutException; +import io.netty.handler.timeout.ReadTimeoutHandler; +import io.netty.util.ReferenceCountUtil; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.FetcherState; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.pullserver.PullServerConstants; +import org.apache.tajo.pullserver.PullServerConstants.Param; +import org.apache.tajo.pullserver.PullServerUtil; +import org.apache.tajo.pullserver.PullServerUtil.PullServerParams; +import org.apache.tajo.pullserver.PullServerUtil.PullServerRequestURIBuilder; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.pullserver.retriever.FileChunkMeta; +import org.apache.tajo.rpc.NettyUtils; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.storage.StorageUtil; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +public class LocalFetcher extends AbstractFetcher { + + private final static Log LOG = LogFactory.getLog(LocalFetcher.class); + +// private final ExecutionBlockContext executionBlockContext; + private final TajoPullServerService pullServerService; + + private final String host; + private int port; + private final Bootstrap bootstrap; + private final int maxUrlLength; + private final List chunkMetas = new ArrayList<>(); + private final String tableName; + private final FileSystem localFileSystem; + private final LocalDirAllocator localDirAllocator; + + @VisibleForTesting + public LocalFetcher(TajoConf conf, URI uri, String tableName) throws IOException { + super(conf, uri); + this.maxUrlLength = conf.getIntVar(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH); + this.tableName = tableName; + this.localFileSystem = new LocalFileSystem(); + this.localDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); + this.pullServerService = null; + + String scheme = uri.getScheme() == null ? "http" : uri.getScheme(); + this.host = uri.getHost() == null ? "localhost" : uri.getHost(); + this.port = uri.getPort(); + if (port == -1) { + if (scheme.equalsIgnoreCase("http")) { + this.port = 80; + } else if (scheme.equalsIgnoreCase("https")) { + this.port = 443; + } + } + + bootstrap = new Bootstrap() + .group( + NettyUtils.getSharedEventLoopGroup(NettyUtils.GROUP.FETCHER, + conf.getIntVar(ConfVars.SHUFFLE_RPC_CLIENT_WORKER_THREAD_NUM))) + .channel(NioSocketChannel.class) + .option(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, + conf.getIntVar(ConfVars.SHUFFLE_FETCHER_CONNECT_TIMEOUT) * 1000) + .option(ChannelOption.SO_RCVBUF, 1048576) // set 1M + .option(ChannelOption.TCP_NODELAY, true); + } + + public LocalFetcher(TajoConf conf, URI uri, ExecutionBlockContext executionBlockContext, String tableName) { + super(conf, uri); + this.localFileSystem = executionBlockContext.getLocalFS(); + this.localDirAllocator = executionBlockContext.getLocalDirAllocator(); + this.maxUrlLength = conf.getIntVar(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH); + this.tableName = tableName; + + Optional optional = executionBlockContext.getSharedResource().getPullServerService(); + if (optional.isPresent()) { + // local pull server service + this.pullServerService = optional.get(); + this.host = null; + this.bootstrap = null; + + } else if (PullServerUtil.useExternalPullServerService(conf)) { + // external pull server service + pullServerService = null; + + String scheme = uri.getScheme() == null ? "http" : uri.getScheme(); + this.host = uri.getHost() == null ? "localhost" : uri.getHost(); + this.port = uri.getPort(); + if (port == -1) { + if (scheme.equalsIgnoreCase("http")) { + this.port = 80; + } else if (scheme.equalsIgnoreCase("https")) { + this.port = 443; + } + } + + bootstrap = new Bootstrap() + .group( + NettyUtils.getSharedEventLoopGroup(NettyUtils.GROUP.FETCHER, + conf.getIntVar(ConfVars.SHUFFLE_RPC_CLIENT_WORKER_THREAD_NUM))) + .channel(NioSocketChannel.class) + .option(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, + conf.getIntVar(ConfVars.SHUFFLE_FETCHER_CONNECT_TIMEOUT) * 1000) + .option(ChannelOption.SO_RCVBUF, 1048576) // set 1M + .option(ChannelOption.TCP_NODELAY, true); + } else { + endFetch(FetcherState.FETCH_FAILED); + throw new TajoInternalError("Pull server service is not initialized"); + } + } + + @Override + public List get() throws IOException { + return pullServerService != null ? getDirect() : getFromFetchURI(); + } + + private List getDirect() throws IOException { + final List fileChunks = new ArrayList<>(); + startTime = System.currentTimeMillis(); + PullServerParams params = new PullServerParams(uri.toString()); + try { + fileChunks.addAll(pullServerService.getFileChunks(conf, localDirAllocator, params)); + } catch (ExecutionException e) { + endFetch(FetcherState.FETCH_FAILED); + throw new TajoInternalError(e); + } + fileChunks.stream().forEach(c -> c.setEbId(tableName)); + endFetch(FetcherState.FETCH_DATA_FINISHED); + fileLen = fileChunks.get(0).getFile().length(); + return fileChunks; + } + + private List getFromFetchURI() throws IOException { + final PullServerParams params = new PullServerParams(uri.toString()); + final Path queryBaseDir = PullServerUtil.getBaseOutputDir(params.queryId(), params.ebId()); + + if (PullServerUtil.isRangeShuffle(params.shuffleType())) { + return getChunksForRangeShuffle(params, queryBaseDir); + } else if (PullServerUtil.isHashShuffle(params.shuffleType())) { + return getChunksForHashShuffle(params, queryBaseDir); + } else { + endFetch(FetcherState.FETCH_FAILED); + throw new IllegalArgumentException("unknown shuffle type: " + params.shuffleType()); + } + } + + private List getChunksForHashShuffle(final PullServerParams params, final Path queryBaseDir) + throws IOException { + final List fileChunks = new ArrayList<>(); + final String partId = params.partId(); + final long offset = params.offset(); + final long length = params.length(); + final int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); + final Path partPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); + + if (!localDirAllocator.ifExists(partPath.toString(), conf)) { + endFetch(FetcherState.FETCH_FAILED); + throw new IOException("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); + } + final Path path = localFileSystem.makeQualified(localDirAllocator.getLocalPathToRead(partPath.toString(), conf)); + final File file = new File(path.toUri()); + final long startPos = (offset >= 0 && length >= 0) ? offset : 0; + final long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + endFetch(FetcherState.FETCH_FAILED); + throw new IOException("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); + } + if (readLen > 0) { + final FileChunk chunk = new FileChunk(file, startPos, readLen); + chunk.setEbId(tableName); + chunk.setFromRemote(false); + fileChunks.add(chunk); + } + + endFetch(FetcherState.FETCH_DATA_FINISHED); + return fileChunks; + } + + private List getChunksForRangeShuffle(final PullServerParams params, final Path queryBaseDir) + throws IOException { + final List fileChunks = new ArrayList<>(); + + if (state == FetcherState.FETCH_INIT) { + final ChannelInitializer initializer = new HttpClientChannelInitializer(); + bootstrap.handler(initializer); + } + + this.startTime = System.currentTimeMillis(); + this.state = FetcherState.FETCH_META_FETCHING; + ChannelFuture future = null; + try { + future = bootstrap.clone().connect(new InetSocketAddress(host, port)) + .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE); + + // Wait until the connection attempt succeeds or fails. + Channel channel = future.awaitUninterruptibly().channel(); + if (!future.isSuccess()) { + endFetch(FetcherState.FETCH_FAILED); + throw new IOException(future.cause()); + } + + for (URI eachURI : createChunkMetaRequestURIs(host, port, params)) { + String query = eachURI.getPath() + + (eachURI.getRawQuery() != null ? "?" + eachURI.getRawQuery() : ""); + HttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, query); + request.headers().set(HttpHeaders.Names.HOST, host); + request.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE); + request.headers().set(HttpHeaders.Names.ACCEPT_ENCODING, HttpHeaders.Values.GZIP); + + if(LOG.isDebugEnabled()) { + LOG.debug("Status: " + getState() + ", URI:" + eachURI); + } + // Send the HTTP request. + channel.writeAndFlush(request); + } + // Wait for the server to close the connection. throw exception if failed + channel.closeFuture().syncUninterruptibly(); + + state = FetcherState.FETCH_DATA_FETCHING; + for (FileChunkMeta eachMeta : chunkMetas) { + Path outputPath = StorageUtil.concatPath(queryBaseDir, eachMeta.getTaskId(), "output"); + if (!localDirAllocator.ifExists(outputPath.toString(), conf)) { + LOG.warn("Range shuffle - file not exist. " + outputPath); + continue; + } + Path path = localFileSystem.makeQualified(localDirAllocator.getLocalPathToRead(outputPath.toString(), conf)); + FileChunk chunk = new FileChunk(new File(URI.create(path.toUri() + "/output")), + eachMeta.getStartOffset(), eachMeta.getLength()); + chunk.setEbId(tableName); + fileChunks.add(chunk); + } + + return fileChunks; + } finally { + if(future != null && future.channel().isOpen()){ + // Close the channel to exit. + future.channel().close().awaitUninterruptibly(); + } + + endFetch(FetcherState.FETCH_DATA_FINISHED); + } + } + + public class HttpClientHandler extends ChannelInboundHandlerAdapter { + private int length = -1; + private int totalReceivedContentLength = 0; + private byte[] buf; + private final Gson gson = new Gson(); + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) + throws Exception { + + messageReceiveCount++; + if (msg instanceof HttpResponse) { + try { + HttpResponse response = (HttpResponse) msg; + + StringBuilder sb = new StringBuilder(); + if (LOG.isDebugEnabled()) { + sb.append("STATUS: ").append(response.getStatus()).append(", VERSION: ") + .append(response.getProtocolVersion()).append(", HEADER: "); + } + if (!response.headers().names().isEmpty()) { + for (String name : response.headers().names()) { + for (String value : response.headers().getAll(name)) { + if (LOG.isDebugEnabled()) { + sb.append(name).append(" = ").append(value); + } + if (this.length == -1 && name.equals("Content-Length")) { + this.length = Integer.parseInt(value); + if (buf == null || buf.length < this.length) { + buf = new byte[this.length]; + } + } + } + } + } + if (LOG.isDebugEnabled()) { + LOG.debug(sb.toString()); + } + + if (response.getStatus().code() == HttpResponseStatus.NO_CONTENT.code()) { + LOG.warn("There are no data corresponding to the request"); + length = 0; + return; + } else if (response.getStatus().code() != HttpResponseStatus.OK.code()) { + LOG.error(response.getStatus().reasonPhrase()); + state = TajoProtos.FetcherState.FETCH_FAILED; + return; + } + } catch (Exception e) { + LOG.error(e.getMessage(), e); + } finally { + ReferenceCountUtil.release(msg); + } + } + + if (msg instanceof HttpContent) { + HttpContent httpContent = (HttpContent) msg; + ByteBuf content = httpContent.content(); + + if (state != FetcherState.FETCH_FAILED) { + try { + if (content.isReadable()) { + int contentLength = content.readableBytes(); + if ((totalReceivedContentLength + contentLength) == length) { + state = FetcherState.FETCH_META_FINISHED; + } + content.readBytes(buf, totalReceivedContentLength, contentLength); + totalReceivedContentLength += contentLength; + if (state.equals(FetcherState.FETCH_META_FINISHED)) { + List jsonMetas = gson.fromJson(new String(buf), List.class); + for (String eachJson : jsonMetas) { + FileChunkMeta meta = gson.fromJson(eachJson, FileChunkMeta.class); + chunkMetas.add(meta); + } + totalReceivedContentLength = 0; + length = -1; + } else if (totalReceivedContentLength > length) { + // TODO + throw new IOException("Illegal length: " + totalReceivedContentLength + ", expected length: " + length); + } + } + } catch (Exception e) { + LOG.error(e.getMessage(), e); + } finally { + ReferenceCountUtil.release(msg); + } + } else { + // http content contains the reason why the fetch failed. + LOG.error(content.toString(Charset.defaultCharset())); + } + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) + throws Exception { + if (cause instanceof ReadTimeoutException) { + LOG.warn(cause.getMessage(), cause); + } else { + LOG.error("Fetch failed :", cause); + } + + // this fetching will be retry + finishTime = System.currentTimeMillis(); + state = TajoProtos.FetcherState.FETCH_FAILED; + ctx.close(); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + if(getState() == FetcherState.FETCH_INIT || getState() == FetcherState.FETCH_META_FETCHING){ + //channel is closed, but cannot complete fetcher + finishTime = System.currentTimeMillis(); + LOG.error("Channel closed by peer: " + ctx.channel()); + state = TajoProtos.FetcherState.FETCH_FAILED; + } + + super.channelUnregistered(ctx); + } + } + + public class HttpClientChannelInitializer extends ChannelInitializer { + + @Override + protected void initChannel(Channel channel) throws Exception { + ChannelPipeline pipeline = channel.pipeline(); + + int maxChunkSize = conf.getIntVar(ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE); + int readTimeout = conf.getIntVar(ConfVars.SHUFFLE_FETCHER_READ_TIMEOUT); + + pipeline.addLast("codec", new HttpClientCodec(4096, 8192, maxChunkSize)); + pipeline.addLast("inflater", new HttpContentDecompressor()); + pipeline.addLast("timeout", new ReadTimeoutHandler(readTimeout, TimeUnit.SECONDS)); + pipeline.addLast("handler", new HttpClientHandler()); + } + } + + private List createChunkMetaRequestURIs(String pullServerAddr, int pullServerPort, PullServerParams params) { + final PullServerRequestURIBuilder builder = new PullServerRequestURIBuilder(pullServerAddr, pullServerPort, maxUrlLength); + builder.setRequestType(PullServerConstants.META_REQUEST_PARAM_STRING) + .setQueryId(params.queryId()) + .setShuffleType(params.shuffleType()) + .setEbId(params.ebId()) + .setPartId(params.partId()); + + if (params.contains(Param.OFFSET)) { + builder.setOffset(params.offset()).setLength(params.length()); + } + + if (PullServerUtil.isRangeShuffle(params.shuffleType())) { + builder.setStartKeyBase64(params.startKey()) + .setEndKeyBase64(params.endKey()) + .setLastInclude(params.last()); + } + + if (params.contains(Param.TASK_ID)) { + builder.setTaskAttemptIds(params.taskAttemptIds()); + } + return builder.build(true); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java similarity index 71% rename from tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java rename to tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java index 250b4cc7c8..0e846126ff 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java @@ -33,7 +33,7 @@ import org.apache.tajo.TajoProtos; import org.apache.tajo.TajoProtos.FetcherState; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.PullServerConstants; import org.apache.tajo.pullserver.retriever.FileChunk; import org.apache.tajo.rpc.NettyUtils; @@ -44,6 +44,7 @@ import java.net.InetSocketAddress; import java.net.URI; import java.nio.channels.FileChannel; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; @@ -52,33 +53,18 @@ * Fetcher fetches data from a given uri via HTTP protocol and stores them into * a specific file. It aims at asynchronous and efficient data transmit. */ -public class Fetcher { +public class RemoteFetcher extends AbstractFetcher { - private final static Log LOG = LogFactory.getLog(Fetcher.class); - - private final URI uri; - private final FileChunk fileChunk; - private final TajoConf conf; + private final static Log LOG = LogFactory.getLog(RemoteFetcher.class); private final String host; private int port; - private final boolean useLocalFile; - - private long startTime; - private volatile long finishTime; - private long fileLen; - private int messageReceiveCount; - private TajoProtos.FetcherState state; - private Bootstrap bootstrap; - private List chunkLengths = new ArrayList<>(); + private final Bootstrap bootstrap; + private final List chunkLengths = new ArrayList<>(); - public Fetcher(TajoConf conf, URI uri, FileChunk chunk) { - this.uri = uri; - this.fileChunk = chunk; - this.useLocalFile = !chunk.fromRemote(); - this.state = TajoProtos.FetcherState.FETCH_INIT; - this.conf = conf; + public RemoteFetcher(TajoConf conf, URI uri, FileChunk chunk) { + super(conf, uri, chunk); String scheme = uri.getScheme() == null ? "http" : uri.getScheme(); this.host = uri.getHost() == null ? "localhost" : uri.getHost(); @@ -91,50 +77,21 @@ public Fetcher(TajoConf conf, URI uri, FileChunk chunk) { } } - if (!useLocalFile) { - bootstrap = new Bootstrap() - .group( - NettyUtils.getSharedEventLoopGroup(NettyUtils.GROUP.FETCHER, - conf.getIntVar(TajoConf.ConfVars.SHUFFLE_RPC_CLIENT_WORKER_THREAD_NUM))) - .channel(NioSocketChannel.class) - .option(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, - conf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_CONNECT_TIMEOUT) * 1000) - .option(ChannelOption.SO_RCVBUF, 1048576) // set 1M - .option(ChannelOption.TCP_NODELAY, true); - } - } - - public long getStartTime() { - return startTime; - } - - public long getFinishTime() { - return finishTime; - } - - public long getFileLen() { - return fileLen; - } - - public TajoProtos.FetcherState getState() { - return state; - } - - public int getMessageReceiveCount() { - return messageReceiveCount; + bootstrap = new Bootstrap() + .group( + NettyUtils.getSharedEventLoopGroup(NettyUtils.GROUP.FETCHER, + conf.getIntVar(TajoConf.ConfVars.SHUFFLE_RPC_CLIENT_WORKER_THREAD_NUM))) + .channel(NioSocketChannel.class) + .option(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, + conf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_CONNECT_TIMEOUT) * 1000) + .option(ChannelOption.SO_RCVBUF, 1048576) // set 1M + .option(ChannelOption.TCP_NODELAY, true); } + @Override public List get() throws IOException { List fileChunks = new ArrayList<>(); - if (useLocalFile) { - startTime = System.currentTimeMillis(); - finishTime = System.currentTimeMillis(); - state = TajoProtos.FetcherState.FETCH_FINISHED; - fileChunks.add(fileChunk); - fileLen = fileChunk.getFile().length(); - return fileChunks; - } if (state == FetcherState.FETCH_INIT) { ChannelInitializer initializer = new HttpClientChannelInitializer(fileChunk.getFile()); @@ -142,7 +99,7 @@ public List get() throws IOException { } this.startTime = System.currentTimeMillis(); - this.state = TajoProtos.FetcherState.FETCH_FETCHING; + this.state = FetcherState.FETCH_DATA_FETCHING; ChannelFuture future = null; try { future = bootstrap.clone().connect(new InetSocketAddress(host, port)) @@ -179,7 +136,7 @@ public List get() throws IOException { if (eachChunkLength == 0) continue; FileChunk chunk = new FileChunk(fileChunk.getFile(), start, eachChunkLength); chunk.setEbId(fileChunk.getEbId()); - chunk.setFromRemote(fileChunk.fromRemote()); + chunk.setFromRemote(true); fileChunks.add(chunk); start += eachChunkLength; } @@ -206,15 +163,12 @@ public List get() throws IOException { } } - public URI getURI() { - return this.uri; - } - - class HttpClientHandler extends ChannelInboundHandlerAdapter { + public class HttpClientHandler extends ChannelInboundHandlerAdapter { private final File file; private RandomAccessFile raf; private FileChannel fc; private long length = -1; + private int totalReceivedContentLength = 0; public HttpClientHandler(File file) throws FileNotFoundException { this.file = file; @@ -247,8 +201,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) } } } - if (response.headers().contains(TajoPullServerService.CHUNK_LENGTH_HEADER_NAME)) { - String stringOffset = response.headers().get(TajoPullServerService.CHUNK_LENGTH_HEADER_NAME); + if (response.headers().contains(PullServerConstants.CHUNK_LENGTH_HEADER_NAME)) { + String stringOffset = response.headers().get(PullServerConstants.CHUNK_LENGTH_HEADER_NAME); for (String eachSplit : stringOffset.split(",")) { chunkLengths.add(Long.parseLong(eachSplit)); @@ -264,8 +218,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) length = 0; return; } else if (response.getStatus().code() != HttpResponseStatus.OK.code()) { - LOG.error(response.getStatus().reasonPhrase()); - state = TajoProtos.FetcherState.FETCH_FAILED; + LOG.error(response.getStatus().reasonPhrase(), response.getDecoderResult().cause()); + endFetch(FetcherState.FETCH_FAILED); return; } } catch (Exception e) { @@ -276,29 +230,38 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) } if (msg instanceof HttpContent) { - try { - HttpContent httpContent = (HttpContent) msg; - ByteBuf content = httpContent.content(); - if (content.isReadable()) { - content.readBytes(fc, content.readableBytes()); - } - - if (msg instanceof LastHttpContent) { - if (raf != null) { - fileLen = file.length(); + HttpContent httpContent = (HttpContent) msg; + ByteBuf content = httpContent.content(); + + if (state != FetcherState.FETCH_FAILED) { + try { + if (content.isReadable()) { + totalReceivedContentLength += content.readableBytes(); + content.readBytes(fc, content.readableBytes()); } - finishTime = System.currentTimeMillis(); - if (state != TajoProtos.FetcherState.FETCH_FAILED) { - state = TajoProtos.FetcherState.FETCH_FINISHED; - } + // TODO: check compatability with yarn pull server + if (msg instanceof LastHttpContent) { + if (raf != null) { + fileLen = file.length(); + } - IOUtils.cleanup(LOG, fc, raf); + if (totalReceivedContentLength == length) { + endFetch(FetcherState.FETCH_DATA_FINISHED); + } else { + endFetch(FetcherState.FETCH_FAILED); + throw new IOException("Invalid fetch length: " + totalReceivedContentLength + ", but expected " + length); + } + IOUtils.cleanup(LOG, fc, raf); + } + } catch (Exception e) { + LOG.error(e.getMessage(), e); + } finally { + ReferenceCountUtil.release(msg); } - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } finally { - ReferenceCountUtil.release(msg); + } else { + // http content contains the reason why the fetch failed. + LOG.error(content.toString(Charset.defaultCharset())); } } } @@ -314,26 +277,24 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) // this fetching will be retry IOUtils.cleanup(LOG, fc, raf); - finishTime = System.currentTimeMillis(); - state = TajoProtos.FetcherState.FETCH_FAILED; + endFetch(FetcherState.FETCH_FAILED); ctx.close(); } @Override public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { - if(getState() != TajoProtos.FetcherState.FETCH_FINISHED){ + if(getState() != FetcherState.FETCH_DATA_FINISHED){ //channel is closed, but cannot complete fetcher - finishTime = System.currentTimeMillis(); + endFetch(FetcherState.FETCH_FAILED); LOG.error("Channel closed by peer: " + ctx.channel()); - state = TajoProtos.FetcherState.FETCH_FAILED; } IOUtils.cleanup(LOG, fc, raf); - + super.channelUnregistered(ctx); } } - class HttpClientChannelInitializer extends ChannelInitializer { + public class HttpClientChannelInitializer extends ChannelInitializer { private final File file; public HttpClientChannelInitializer(File file) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index de337aed3d..f78404a756 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -37,6 +37,7 @@ import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.metrics.Node; import org.apache.tajo.plan.function.python.PythonScriptEngine; +import org.apache.tajo.pullserver.PullServerUtil; import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.querymaster.QueryMaster; import org.apache.tajo.querymaster.QueryMasterManagerService; @@ -64,7 +65,6 @@ import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; import java.util.ArrayList; -import java.util.HashMap; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.tajo.conf.TajoConf.ConfVars; @@ -144,7 +144,17 @@ public void serviceInit(Configuration conf) throws Exception { queryMasterManagerService = new QueryMasterManagerService(workerContext, qmManagerPort); addIfService(queryMasterManagerService); - this.taskManager = new TaskManager(dispatcher, workerContext); + if (!PullServerUtil.useExternalPullServerService(systemConf)) { + pullService = new TajoPullServerService(); + addIfService(pullService); + } + +// if(!TajoPullServerService.isStandalone()) { +// pullService = new TajoPullServerService(); +// addIfService(pullService); +// } + + this.taskManager = new TaskManager(dispatcher, workerContext, pullService); addService(taskManager); this.taskExecutor = new TaskExecutor(workerContext); @@ -158,21 +168,16 @@ public void serviceInit(Configuration conf) throws Exception { addService(new NodeStatusUpdater(workerContext)); int httpPort = 0; - if(!TajoPullServerService.isStandalone()) { - pullService = new TajoPullServerService(); - addIfService(pullService); - } - if (!systemConf.getBoolVar(ConfVars.$TEST_MODE)) { httpPort = initWebServer(); } super.serviceInit(conf); - int pullServerPort; + int pullServerPort = systemConf.getIntVar(ConfVars.PULLSERVER_PORT); if(pullService != null){ pullServerPort = pullService.getPort(); - } else { + } else if (TajoPullServerService.isStandalone()) { pullServerPort = getStandAlonePullServerPort(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index 19d5da41d3..4a5b0b4747 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -54,5 +54,5 @@ public interface Task { TaskHistory createTaskHistory(); - List getFetchers(); + List getFetchers(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java index 55eb02ab15..753b4abca1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.FetcherState; import org.apache.tajo.TajoProtos.TaskAttemptState; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.Schema; @@ -53,7 +54,6 @@ import org.apache.tajo.plan.serder.PlanProto.EnforceProperty.EnforceType; import org.apache.tajo.plan.serder.PlanProto.ShuffleType; import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.pullserver.retriever.FileChunk; import org.apache.tajo.querymaster.Repartitioner; import org.apache.tajo.rpc.NullCallback; @@ -84,7 +84,7 @@ public class TaskImpl implements Task { private final Path taskDir; private final TaskAttemptContext context; - private List fetcherRunners; + private List fetcherRunners; private LogicalNode plan; private PhysicalExec executor; @@ -269,7 +269,7 @@ public void fetch(ExecutorService fetcherExecutor) { return taskIdStr1.compareTo(taskIdStr2); }); - for (Fetcher f : fetcherRunners) { + for (AbstractFetcher f : fetcherRunners) { fetcherExecutor.submit(new FetchRunner(context, f)); } } @@ -516,14 +516,14 @@ public TaskHistory createTaskHistory() { taskHistory.setTotalFetchCount(fetcherRunners.size()); int i = 0; FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); - for (Fetcher fetcher : fetcherRunners) { + for (AbstractFetcher fetcher : fetcherRunners) { builder.setStartTime(fetcher.getStartTime()); builder.setFinishTime(fetcher.getFinishTime()); builder.setFileLength(fetcher.getFileLen()); builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); builder.setState(fetcher.getState()); taskHistory.addFetcherHistory(builder.build()); - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; + if (fetcher.getState() == FetcherState.FETCH_DATA_FINISHED) i++; } taskHistory.setFinishedFetchCount(i); } @@ -534,7 +534,7 @@ public TaskHistory createTaskHistory() { return taskHistory; } - public List getFetchers() { + public List getFetchers() { return fetcherRunners; } @@ -585,10 +585,10 @@ private FileFragment[] localizeFetchedData(String name) private class FetchRunner implements Runnable { private final TaskAttemptContext ctx; - private final Fetcher fetcher; + private final AbstractFetcher fetcher; private int maxRetryNum; - public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { + public FetchRunner(TaskAttemptContext ctx, AbstractFetcher fetcher) { this.ctx = ctx; this.fetcher = fetcher; this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); @@ -612,7 +612,7 @@ public void run() { } try { List fetched = fetcher.get(); - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) { + if (fetcher.getState() == FetcherState.FETCH_DATA_FINISHED) { for (FileChunk eachFetch : fetched) { if (eachFetch.getFile() != null) { if (!eachFetch.fromRemote()) { @@ -630,7 +630,7 @@ public void run() { retryNum++; } } finally { - if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ + if(fetcher.getState() == FetcherState.FETCH_DATA_FINISHED){ fetcherFinished(ctx); } else { if (retryNum == maxRetryNum) { @@ -669,8 +669,8 @@ private synchronized void fetcherFinished(TaskAttemptContext ctx) { } } - private List getFetchRunners(TaskAttemptContext ctx, - List fetches) throws IOException { + private List getFetchRunners(TaskAttemptContext ctx, + List fetches) throws IOException { if (fetches.size() > 0) { Path inputDir = executionBlockContext.getLocalDirAllocator(). @@ -681,7 +681,7 @@ private List getFetchRunners(TaskAttemptContext ctx, File storeDir; File defaultStoreFile; List storeChunkList = new ArrayList<>(); - List runnerList = Lists.newArrayList(); + List runnerList = Lists.newArrayList(); for (FetchProto f : fetches) { storeDir = new File(inputDir.toString(), f.getName()); @@ -696,44 +696,48 @@ private List getFetchRunners(TaskAttemptContext ctx, WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { - - List localChunkCandidates = getLocalStoredFileChunk(uri, systemConf); - - for (FileChunk localChunk : localChunkCandidates) { - // When a range request is out of range, storeChunk will be NULL. This case is normal state. - // So, we should skip and don't need to create storeChunk. - if (localChunk == null || localChunk.length() == 0) { - continue; - } - - if (localChunk.getFile() != null && localChunk.startOffset() > -1) { - localChunk.setFromRemote(false); - localStoreChunkCount++; - } else { - localChunk = new FileChunk(defaultStoreFile, 0, -1); - localChunk.setFromRemote(true); - } - localChunk.setEbId(f.getName()); - storeChunkList.add(localChunk); - } + localStoreChunkCount++; + runnerList.add(new LocalFetcher(systemConf, uri, executionBlockContext, f.getName())); + +// List localChunkCandidates = getLocalStoredFileChunk(uri, systemConf); +// +// for (FileChunk localChunk : localChunkCandidates) { +// // When a range request is out of range, storeChunk will be NULL. This case is normal state. +// // So, we should skip and don't need to create storeChunk. +// if (localChunk == null || localChunk.length() == 0) { +// continue; +// } +// +// if (localChunk.getFile() != null && localChunk.startOffset() > -1) { +// localChunk.setFromRemote(false); +// localStoreChunkCount++; +// } else { +// localChunk = new FileChunk(defaultStoreFile, 0, -1); +// localChunk.setFromRemote(true); +// } +// localChunk.setEbId(f.getName()); +// storeChunkList.add(localChunk); +// } } else { FileChunk remoteChunk = new FileChunk(defaultStoreFile, 0, -1); remoteChunk.setFromRemote(true); remoteChunk.setEbId(f.getName()); - storeChunkList.add(remoteChunk); +// storeChunkList.add(remoteChunk); + runnerList.add(new RemoteFetcher(systemConf, uri, remoteChunk)); + i++; } // If we decide that intermediate data should be really fetched from a remote host, storeChunk // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it - for (FileChunk eachChunk : storeChunkList) { - Fetcher fetcher = new Fetcher(systemConf, uri, eachChunk); - runnerList.add(fetcher); - i++; - if (LOG.isDebugEnabled()) { - LOG.debug("Create a new Fetcher with storeChunk:" + eachChunk.toString()); - } - } +// for (FileChunk eachChunk : storeChunkList) { +// RemoteFetcher fetcher = new RemoteFetcher(systemConf, uri, eachChunk); +// runnerList.add(fetcher); +// i++; +// if (LOG.isDebugEnabled()) { +// LOG.debug("Create a new Fetcher with storeChunk:" + eachChunk.toString()); +// } +// } } } ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); @@ -745,96 +749,9 @@ private List getFetchRunners(TaskAttemptContext ctx, } } - private List getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { - // Parse the URI - - // Parsing the URL into key-values - final Map> params = TajoPullServerService.decodeParams(fetchURI.toString()); - - String partId = params.get("p").get(0); - String queryId = params.get("qid").get(0); - String shuffleType = params.get("type").get(0); - String sid = params.get("sid").get(0); - - final List taskIdList = params.get("ta"); - final List offsetList = params.get("offset"); - final List lengthList = params.get("length"); - - long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; - long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; - - if (LOG.isDebugEnabled()) { - LOG.debug("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId - + ", taskIds=" + taskIdList); - } - - // The working directory of Tajo worker for each query, including stage - Path queryBaseDir = TajoPullServerService.getBaseOutputDir(queryId, sid); - - List chunkList = new ArrayList<>(); - // If the stage requires a range shuffle - if (shuffleType.equals("r")) { - - final String startKey = params.get("start").get(0); - final String endKey = params.get("end").get(0); - final boolean last = params.get("final") != null; - final List taskIds = TajoPullServerService.splitMaps(taskIdList); - - long before = System.currentTimeMillis(); - for (String eachTaskId : taskIds) { - Path outputPath = StorageUtil.concatPath(queryBaseDir, eachTaskId, "output"); - if (!executionBlockContext.getLocalDirAllocator().ifExists(outputPath.toString(), conf)) { - LOG.warn("Range shuffle - file not exist. " + outputPath); - continue; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(outputPath.toString(), conf)); - - try { - FileChunk chunk = TajoPullServerService.getFileChunks(queryId, sid, path, startKey, endKey, last); - chunkList.add(chunk); - } catch (Throwable t) { - LOG.error(t.getMessage(), t); - throw new IOException(t.getCause()); - } - } - long after = System.currentTimeMillis(); - if (LOG.isDebugEnabled()) { - LOG.debug("Index lookup time: " + (after - before) + " ms"); - } - - // If the stage requires a hash shuffle or a scattered hash shuffle - } else if (shuffleType.equals("h") || shuffleType.equals("s")) { - int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); - Path partPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); - - if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath.toString(), conf)) { - throw new IOException("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath.toString(), conf)); - File file = new File(path.toUri()); - long startPos = (offset >= 0 && length >= 0) ? offset : 0; - long readLen = (offset >= 0 && length >= 0) ? length : file.length(); - - if (startPos >= file.length()) { - throw new IOException("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); - } - FileChunk chunk = new FileChunk(file, startPos, readLen); - chunkList.add(chunk); - - } else { - throw new IOException("Unknown shuffle type"); - } - - return chunkList; - } - public static Path getTaskAttemptDir(TaskAttemptId quid) { - Path workDir = - StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), - String.valueOf(quid.getTaskId().getId()), - String.valueOf(quid.getId())); - return workDir; + return StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), + String.valueOf(quid.getTaskId().getId()), + String.valueOf(quid.getId())); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index 9e2e9e80f2..d4160aab43 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -30,6 +30,7 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.rpc.AsyncRpcClient; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.rpc.RpcClientManager; @@ -42,6 +43,7 @@ import java.net.InetSocketAddress; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import static org.apache.tajo.ResourceProtos.*; @@ -57,12 +59,19 @@ public class TaskManager extends AbstractService implements EventHandlertajo-pullserver ${project.version} + + org.apache.tajo + tajo-yarn + ${project.version} + org.apache.tajo tajo-client @@ -989,13 +994,6 @@ ${hadoop.version} test - - org.apache.hadoop - hadoop-yarn-server-tests - ${hadoop.version} - test-jar - test - org.apache.hadoop hadoop-mapreduce-client-shuffle diff --git a/tajo-pullserver/pom.xml b/tajo-pullserver/pom.xml index 2a1a745bb3..6c805ac09c 100644 --- a/tajo-pullserver/pom.xml +++ b/tajo-pullserver/pom.xml @@ -56,12 +56,12 @@ org.apache.tajo - tajo-rpc-protobuf + tajo-catalog-common + provided org.apache.tajo - tajo-catalog-common - provided + tajo-rpc-protobuf org.apache.tajo diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java index 9c3c523529..670f406099 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java @@ -24,18 +24,9 @@ public class FileCloseListener implements GenericFutureListener { private FadvisedFileRegion filePart; - private String requestUri; - private TajoPullServerService pullServerService; - private long startTime; - public FileCloseListener(FadvisedFileRegion filePart, - String requestUri, - long startTime, - TajoPullServerService pullServerService) { + public FileCloseListener(FadvisedFileRegion filePart) { this.filePart = filePart; - this.requestUri = requestUri; - this.pullServerService = pullServerService; - this.startTime = startTime; } // TODO error handling; distinguish IO/connection failures, @@ -46,8 +37,8 @@ public void operationComplete(ChannelFuture future) { filePart.transferSuccessful(); } filePart.deallocate(); - if (pullServerService != null) { - pullServerService.completeFileChunk(filePart, requestUri, startTime); - } +// if (pullServerService != null) { +// pullServerService.completeFileChunk(filePart, requestUri, startTime); +// } } } diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerConstants.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerConstants.java new file mode 100644 index 0000000000..74f96e7b0b --- /dev/null +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerConstants.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.pullserver; + +public class PullServerConstants { + + /** + * Pull server query parameters + */ + public enum Param { + // Common params + REQUEST_TYPE("rtype"), // can be one of 'm' for meta and 'c' for chunk. + SHUFFLE_TYPE("stype"), // can be one of 'r', 'h', and 's'. + QUERY_ID("qid"), + EB_ID("sid"), + PART_ID("p"), + TASK_ID("ta"), + OFFSET("offset"), + LENGTH("length"), + + // Range shuffle params + START("start"), + END("end"), + FINAL("final"); + + private String key; + + Param(String key) { + this.key = key; + } + + public String key() { + return key; + } + } + + // Request types ---------------------------------------------------------- + + public static final String CHUNK_REQUEST_PARAM_STRING = "c"; + public static final String META_REQUEST_PARAM_STRING = "m"; + + // Shuffle types ---------------------------------------------------------- + + public static final String RANGE_SHUFFLE_PARAM_STRING = "r"; + public static final String HASH_SHUFFLE_PARAM_STRING = "h"; + public static final String SCATTERED_HASH_SHUFFLE_PARAM_STRING = "s"; + + // HTTP header ------------------------------------------------------------ + + public static final String CHUNK_LENGTH_HEADER_NAME = "c"; + + // SSL configurations ----------------------------------------------------- + + public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024; + + public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY = + "tajo.pullserver.ssl.file.buffer.size"; + + // OS cache configurations ------------------------------------------------ + + public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache"; + public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true; + + // Prefetch configurations ------------------------------------------------ + + public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes"; + public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024; + + // Yarn service ID -------------------------------------------------------- + + public static final String PULLSERVER_SERVICEID = "tajo.pullserver"; + + // Standalone pull server ------------------------------------------------- + public static final String PULLSERVER_STANDALONE_ENV_KEY = "TAJO_PULLSERVER_STANDALONE"; + + public static final String PULLSERVER_SERVICE_NAME = "httpshuffle"; +} diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java index c90f1aae9e..f9f665923f 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java @@ -18,13 +18,41 @@ package org.apache.tajo.pullserver; +import com.google.common.base.Preconditions; +import com.google.common.cache.LoadingCache; +import com.google.gson.Gson; +import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang.reflect.MethodUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.nativeio.NativeIO; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.pullserver.PullServerConstants.Param; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.pullserver.retriever.FileChunkMeta; +import org.apache.tajo.pullserver.retriever.IndexCacheKey; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder; +import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader; +import org.apache.tajo.util.Pair; -import java.io.FileDescriptor; +import java.io.*; import java.lang.reflect.Method; +import java.net.URI; +import java.net.URLEncoder; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class PullServerUtil { private static final Log LOG = LogFactory.getLog(PullServerUtil.class); @@ -87,4 +115,640 @@ private static boolean loadNativeIO() { } return loaded; } + + public static Path getBaseOutputDir(String queryId, String executionBlockSequenceId) { + return StorageUtil.concatPath( + queryId, + "output", + executionBlockSequenceId); + } + + public static Path getBaseInputDir(String queryId, String executionBlockId) { + return StorageUtil.concatPath( + queryId, + "in", + executionBlockId); + } + + public static List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList<>(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + + public static boolean isChunkRequest(String requestType) { + return requestType.equals(PullServerConstants.CHUNK_REQUEST_PARAM_STRING); + } + + public static boolean isMetaRequest(String requestType) { + return requestType.equals(PullServerConstants.META_REQUEST_PARAM_STRING); + } + + public static boolean isRangeShuffle(String shuffleType) { + return shuffleType.equals(PullServerConstants.RANGE_SHUFFLE_PARAM_STRING); + } + + public static boolean isHashShuffle(String shuffleType) { + return shuffleType.equals(PullServerConstants.HASH_SHUFFLE_PARAM_STRING) + || shuffleType.equals(PullServerConstants.SCATTERED_HASH_SHUFFLE_PARAM_STRING); + } + + public static class PullServerParams extends HashMap> { + + public PullServerParams(String uri) { + super(new QueryStringDecoder(uri).parameters()); + } + + public boolean contains(Param param) { + return containsKey(param.key()); + } + + public List get(Param param) { + return get(param.key()); + } + + private String checkAndGetFirstParam(Param param) { + Preconditions.checkArgument(contains(param), "Missing " + param.name()); + Preconditions.checkArgument(get(param).size() == 1, "Too many params: " + param.name()); + return get(param).get(0); + } + + private List checkAndGet(Param param) { + Preconditions.checkArgument(contains(param), "Missing " + param.name()); + return get(param); + } + + public String requestType() { + return checkAndGetFirstParam(Param.REQUEST_TYPE); + } + + public String shuffleType() { + return checkAndGetFirstParam(Param.SHUFFLE_TYPE); + } + + public String queryId() { + return checkAndGetFirstParam(Param.QUERY_ID); + } + + public String ebId() { + return checkAndGetFirstParam(Param.EB_ID); + } + + public long offset() { + return contains(Param.OFFSET) && get(Param.OFFSET).size() == 1 ? + Long.parseLong(get(Param.OFFSET).get(0)) : -1L; + } + + public long length() { + return contains(Param.LENGTH) && get(Param.LENGTH).size() == 1 ? + Long.parseLong(get(Param.LENGTH).get(0)) : -1L; + } + + public String startKey() { + return checkAndGetFirstParam(Param.START); + } + + public String endKey() { + return checkAndGetFirstParam(Param.END); + } + + public boolean last() { + return contains(Param.FINAL); + } + + public String partId() { + return checkAndGetFirstParam(Param.PART_ID); + } + + public List taskAttemptIds() { + return checkAndGet(Param.TASK_ID); + } + } + + public static class PullServerRequestURIBuilder { + private final StringBuilder builder = new StringBuilder("http://"); + private String requestType; + private String shuffleType; + private String queryId; + private Integer ebId; + private Integer partId; + private List taskIds; + private List attemptIds; + private List taskAttemptIds; + private Long offset; + private Long length; + private String startKeyBase64; + private String endKeyBase64; + private boolean last; + private final int maxUrlLength; + + public PullServerRequestURIBuilder(String pullServerAddr, int pullServerPort, int maxUrlLength) { + this(pullServerAddr, Integer.toString(pullServerPort), maxUrlLength); + } + + public PullServerRequestURIBuilder(String pullServerAddr, String pullServerPort, int maxUrlLength) { + builder.append(pullServerAddr).append(":").append(pullServerPort).append("/?"); + this.maxUrlLength = maxUrlLength; + } + + public List build(boolean includeTasks) { + append(Param.REQUEST_TYPE, requestType) + .append(Param.QUERY_ID, queryId) + .append(Param.EB_ID, ebId) + .append(Param.PART_ID, partId) + .append(Param.SHUFFLE_TYPE, shuffleType); + + if (startKeyBase64 != null) { + + try { + append(Param.START, URLEncoder.encode(startKeyBase64, "utf-8")) + .append(Param.END, URLEncoder.encode(endKeyBase64, "utf-8")); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + + if (last) { + append(Param.FINAL, Boolean.toString(last)); + } + } + + if (length != null) { + append(Param.OFFSET, offset.toString()) + .append(Param.LENGTH, length.toString()); + } + + List results = new ArrayList<>(); + if (!includeTasks || isHashShuffle(shuffleType)) { + results.add(URI.create(builder.toString())); + } else { + builder.append(Param.TASK_ID.key()).append("="); + List taskAttemptIds = this.taskAttemptIds; + if (taskAttemptIds == null) { + + // Sort task ids to increase cache hit in pull server + taskAttemptIds = IntStream.range(0, taskIds.size()) + .mapToObj(i -> new Pair<>(taskIds.get(i), attemptIds.get(i))) + .sorted((p1, p2) -> p1.getFirst() - p2.getFirst()) + // In the case of hash shuffle each partition has single shuffle file per worker. + // TODO If file is large, consider multiple fetching(shuffle file can be split) + .filter(pair -> pair.getFirst() >= 0) + .map(pair -> pair.getFirst() + "_" + pair.getSecond()) + .collect(Collectors.toList()); + } + + // If the get request is longer than 2000 characters, + // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long. + // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15 + // The below code transforms a long request to multiple requests. + List taskIdsParams = new ArrayList<>(); + StringBuilder taskIdListBuilder = new StringBuilder(); + + boolean first = true; + for (int i = 0; i < taskAttemptIds.size(); i++) { + if (!first) { + taskIdListBuilder.append(","); + } + first = false; + + if (builder.length() + taskIdListBuilder.length() > maxUrlLength) { + taskIdsParams.add(taskIdListBuilder.toString()); + taskIdListBuilder = new StringBuilder(taskAttemptIds.get(i)); + } else { + taskIdListBuilder.append(taskAttemptIds.get(i)); + } + } + // if the url params remain + if (taskIdListBuilder.length() > 0) { + taskIdsParams.add(taskIdListBuilder.toString()); + } + for (String param : taskIdsParams) { + results.add(URI.create(builder + param)); + } + } + + return results; + } + + private PullServerRequestURIBuilder append(Param key, Object val) { + builder.append(key.key()) + .append("=") + .append(val) + .append("&"); + + return this; + } + + public PullServerRequestURIBuilder setRequestType(String type) { + this.requestType = type; + return this; + } + + public PullServerRequestURIBuilder setShuffleType(String shuffleType) { + this.shuffleType = shuffleType; + return this; + } + + public PullServerRequestURIBuilder setQueryId(String queryId) { + this.queryId = queryId; + return this; + } + + public PullServerRequestURIBuilder setEbId(String ebId) { + this.ebId = Integer.parseInt(ebId); + return this; + } + + public PullServerRequestURIBuilder setEbId(Integer ebId) { + this.ebId = ebId; + return this; + } + + public PullServerRequestURIBuilder setPartId(String partId) { + this.partId = Integer.parseInt(partId); + return this; + } + + public PullServerRequestURIBuilder setPartId(Integer partId) { + this.partId = partId; + return this; + } + + public PullServerRequestURIBuilder setTaskIds(List taskIds) { + this.taskIds = taskIds; + return this; + } + + public PullServerRequestURIBuilder setAttemptIds(List attemptIds) { + this.attemptIds = attemptIds; + return this; + } + + public PullServerRequestURIBuilder setTaskAttemptIds(List taskAttemptIds) { + this.taskAttemptIds = taskAttemptIds; + return this; + } + + public PullServerRequestURIBuilder setOffset(long offset) { + this.offset = offset; + return this; + } + + public PullServerRequestURIBuilder setLength(long length) { + this.length = length; + return this; + } + + public PullServerRequestURIBuilder setStartKeyBase64(String startKeyBase64) { + this.startKeyBase64 = startKeyBase64; + return this; + } + + public PullServerRequestURIBuilder setEndKeyBase64(String endKeyBase64) { + this.endKeyBase64 = endKeyBase64; + return this; + } + + public PullServerRequestURIBuilder setLastInclude(boolean last) { + this.last = last; + return this; + } + } + + public static boolean useExternalPullServerService(TajoConf conf) { + // TODO: add more service types like mesos + return TajoPullServerService.isStandalone() + || conf.getBoolVar(ConfVars.YARN_SHUFFLE_SERVICE_ENABLED); + } + + private static FileChunkMeta searchFileChunkMeta(String queryId, + String ebSeqId, + String taskId, + Path outDir, + String startKey, + String endKey, + boolean last, + LoadingCache indexReaderCache, + int lowCacheHitCheckThreshold) throws IOException, ExecutionException { + SearchResult result = searchCorrespondPart(queryId, ebSeqId, outDir, startKey, endKey, last, + indexReaderCache, lowCacheHitCheckThreshold); + // Do not send file chunks of 0 length + if (result != null) { + long startOffset = result.startOffset; + long endOffset = result.endOffset; + + FileChunkMeta chunk = new FileChunkMeta(startOffset, endOffset - startOffset, ebSeqId, taskId); + + if (LOG.isDebugEnabled()) LOG.debug("Retrieve File Chunk: " + chunk); + return chunk; + } else { + return null; + } + } + + private static FileChunk searchFileChunk(String queryId, + String ebSeqId, + Path outDir, + String startKey, + String endKey, + boolean last, + LoadingCache indexReaderCache, + int lowCacheHitCheckThreshold) throws IOException, ExecutionException { + + final SearchResult result = searchCorrespondPart(queryId, ebSeqId, outDir, startKey, endKey, last, + indexReaderCache, lowCacheHitCheckThreshold); + if (result != null) { + long startOffset = result.startOffset; + long endOffset = result.endOffset; + FileChunk chunk = new FileChunk(result.data, startOffset, endOffset - startOffset); + + if (LOG.isDebugEnabled()) LOG.debug("Retrieve File Chunk: " + chunk); + return chunk; + } else { + return null; + } + } + + private static class SearchResult { + File data; + long startOffset; + long endOffset; + + public SearchResult(File data, long startOffset, long endOffset) { + this.data = data; + this.startOffset = startOffset; + this.endOffset = endOffset; + } + } + + private static SearchResult searchCorrespondPart(String queryId, + String ebSeqId, + Path outDir, + String startKey, + String endKey, + boolean last, + LoadingCache indexReaderCache, + int lowCacheHitCheckThreshold) throws IOException, ExecutionException { + BSTIndexReader idxReader = indexReaderCache.get(new IndexCacheKey(outDir, queryId, ebSeqId)); + idxReader.retain(); + + File data; + long startOffset; + long endOffset; + try { + if (LOG.isDebugEnabled()) { + if (indexReaderCache.size() > lowCacheHitCheckThreshold && indexReaderCache.stats().hitRate() < 0.5) { + LOG.debug("Too low cache hit rate: " + indexReaderCache.stats()); + } + } + + Tuple indexedFirst = idxReader.getFirstKey(); + Tuple indexedLast = idxReader.getLastKey(); + + if (indexedFirst == null && indexedLast == null) { // if # of rows is zero + if (LOG.isDebugEnabled()) { + LOG.debug("There is no contents"); + } + return null; + } + + byte[] startBytes = Base64.decodeBase64(startKey); + byte[] endBytes = Base64.decodeBase64(endKey); + + + Tuple start; + Tuple end; + Schema keySchema = idxReader.getKeySchema(); + RowStoreDecoder decoder = RowStoreUtil.createDecoder(keySchema); + + try { + start = decoder.toTuple(startBytes); + } catch (Throwable t) { + throw new IllegalArgumentException("StartKey: " + startKey + + ", decoded byte size: " + startBytes.length, t); + } + + try { + end = decoder.toTuple(endBytes); + } catch (Throwable t) { + throw new IllegalArgumentException("EndKey: " + endKey + + ", decoded byte size: " + endBytes.length, t); + } + + data = new File(URI.create(outDir.toUri() + "/output")); + if (LOG.isDebugEnabled()) { + LOG.debug("GET Request for " + data.getAbsolutePath() + " (start=" + start + ", end=" + end + + (last ? ", last=true" : "") + ")"); + } + + TupleComparator comparator = idxReader.getComparator(); + + if (comparator.compare(end, indexedFirst) < 0 || + comparator.compare(indexedLast, start) < 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("Out of Scope (indexed data [" + indexedFirst + ", " + indexedLast + + "], but request start:" + start + ", end: " + end); + } + return null; + } + + try { + idxReader.init(); + startOffset = idxReader.find(start); + } catch (IOException ioe) { + LOG.error("State Dump (the requested range: " + + "[" + start + ", " + end + ")" + ", idx min: " + + idxReader.getFirstKey() + ", idx max: " + + idxReader.getLastKey()); + throw ioe; + } + try { + endOffset = idxReader.find(end); + if (endOffset == -1) { + endOffset = idxReader.find(end, true); + } + } catch (IOException ioe) { + LOG.error("State Dump (the requested range: " + + "[" + start + ", " + end + ")" + ", idx min: " + + idxReader.getFirstKey() + ", idx max: " + + idxReader.getLastKey()); + throw ioe; + } + + // if startOffset == -1 then case 2-1 or case 3 + if (startOffset == -1) { // this is a hack + // if case 2-1 or case 3 + try { + startOffset = idxReader.find(start, true); + } catch (IOException ioe) { + LOG.error("State Dump (the requested range: " + + "[" + start + ", " + end + ")" + ", idx min: " + + idxReader.getFirstKey() + ", idx max: " + + idxReader.getLastKey()); + throw ioe; + } + } + + if (startOffset == -1) { + throw new IllegalStateException("startOffset " + startOffset + " is negative \n" + + "State Dump (the requested range: " + + "[" + start + ", " + end + ")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: " + + idxReader.getLastKey()); + } + + // if greater than indexed values + if (last || (endOffset == -1 + && comparator.compare(idxReader.getLastKey(), end) < 0)) { + endOffset = data.length(); + } + } finally { + idxReader.release(); + } + + return new SearchResult(data, startOffset, endOffset); + } + + /** + * Retrieve meta information of file chunks which correspond to the requested URI. + * Only meta information for the file chunks which has non-zero length are retrieved. + * + * @param conf + * @param lDirAlloc + * @param localFS + * @param params + * @param gson + * @param indexReaderCache + * @param lowCacheHitCheckThreshold + * @return + * @throws IOException + * @throws ExecutionException + */ + public static List getJsonMeta(final TajoConf conf, + final LocalDirAllocator lDirAlloc, + final FileSystem localFS, + final PullServerParams params, + final Gson gson, + final LoadingCache indexReaderCache, + final int lowCacheHitCheckThreshold) + throws IOException, ExecutionException { + final List taskIds = PullServerUtil.splitMaps(params.taskAttemptIds()); + final Path queryBaseDir = PullServerUtil.getBaseOutputDir(params.queryId(), params.ebId()); + final List jsonMetas = new ArrayList<>(); + + for (String eachTaskId : taskIds) { + Path outputPath = StorageUtil.concatPath(queryBaseDir, eachTaskId, "output"); + if (!lDirAlloc.ifExists(outputPath.toString(), conf)) { + LOG.warn("Range shuffle - file not exist. " + outputPath); + continue; + } + Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(outputPath.toString(), conf)); + FileChunkMeta meta; + meta = PullServerUtil.searchFileChunkMeta(params.queryId(), params.ebId(), eachTaskId, path, + params.startKey(), params.endKey(), params.last(), indexReaderCache, lowCacheHitCheckThreshold); + if (meta != null && meta.getLength() > 0) { + String jsonStr = gson.toJson(meta, FileChunkMeta.class); + jsonMetas.add(jsonStr); + } + } + return jsonMetas; + } + + /** + * Retrieve file chunks which correspond to the requested URI. + * Only the file chunks which has non-zero length are retrieved. + * + * @param conf + * @param lDirAlloc + * @param localFS + * @param params + * @param indexReaderCache + * @param lowCacheHitCheckThreshold + * @return + * @throws IOException + * @throws ExecutionException + */ + public static List getFileChunks(final TajoConf conf, + final LocalDirAllocator lDirAlloc, + final FileSystem localFS, + final PullServerParams params, + final LoadingCache indexReaderCache, + final int lowCacheHitCheckThreshold) + throws IOException, ExecutionException { + final List chunks = new ArrayList<>(); + + final String queryId = params.queryId(); + final String shuffleType = params.shuffleType(); + final String sid = params.ebId(); + + final long offset = params.offset(); + final long length = params.length(); + + final Path queryBaseDir = PullServerUtil.getBaseOutputDir(queryId, sid); + + if (LOG.isDebugEnabled()) { + LOG.debug("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid); + + // the working dir of tajo worker for each query + LOG.debug("PullServer baseDir: " + conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname) + "/" + queryBaseDir); + } + + // if a stage requires a range shuffle + if (PullServerUtil.isRangeShuffle(shuffleType)) { + final List taskIdList = params.taskAttemptIds(); + final List taskIds = PullServerUtil.splitMaps(taskIdList); + + final String startKey = params.startKey(); + final String endKey = params.endKey(); + final boolean last = params.last(); + + long before = System.currentTimeMillis(); + for (String eachTaskId : taskIds) { + Path outputPath = StorageUtil.concatPath(queryBaseDir, eachTaskId, "output"); + if (!lDirAlloc.ifExists(outputPath.toString(), conf)) { + LOG.warn(outputPath + " does not exist."); + continue; + } + Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(outputPath.toString(), conf)); + + FileChunk chunk = PullServerUtil.searchFileChunk(queryId, sid, path, startKey, endKey, last, indexReaderCache, + lowCacheHitCheckThreshold); + if (chunk != null) { + chunks.add(chunk); + } + } + long after = System.currentTimeMillis(); + LOG.info("Index lookup time: " + (after - before) + " ms"); + + // if a stage requires a hash shuffle or a scattered hash shuffle + } else if (PullServerUtil.isHashShuffle(shuffleType)) { + + final String partId = params.partId(); + int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); + Path partPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); + if (!lDirAlloc.ifExists(partPath.toString(), conf)) { + throw new FileNotFoundException(partPath.toString()); + } + + Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(partPath.toString(), conf)); + + File file = new File(path.toUri()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + String errorMessage = "Start pos[" + startPos + "] great than file length [" + file.length() + "]"; + throw new EOFException(errorMessage); + } + FileChunk chunk = new FileChunk(file, startPos, readLen); + chunks.add(chunk); + } else { + throw new IllegalArgumentException(shuffleType); + } + return chunks.stream().filter(c -> c.length() > 0).collect(Collectors.toList()); + } } diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java index 46097120eb..069e660a86 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java @@ -54,11 +54,6 @@ public void startPullServer(TajoConf systemConf) { start(); } - public void start() { - super.start(); - - } - public static void main(String[] args) throws Exception { StringUtils.startupShutdownMessage(TajoPullServerService.PullServer.class, args, LOG); diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index cbeba5287c..770248b203 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -22,8 +22,9 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.cache.RemovalListener; -import com.google.common.collect.Lists; +import com.google.gson.Gson; import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.*; import io.netty.channel.group.ChannelGroup; @@ -31,12 +32,13 @@ import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.handler.codec.http.*; +import io.netty.handler.codec.http.HttpHeaders.Names; +import io.netty.handler.codec.http.HttpHeaders.Values; import io.netty.handler.ssl.SslHandler; import io.netty.handler.stream.ChunkedWriteHandler; import io.netty.util.CharsetUtil; import io.netty.util.concurrent.GenericFutureListener; import io.netty.util.concurrent.GlobalEventExecutor; -import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -56,42 +58,34 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt; import org.apache.hadoop.security.ssl.SSLFactory; import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.catalog.Schema; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.exception.InvalidURLException; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.pullserver.PullServerUtil.PullServerParams; import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.pullserver.retriever.IndexCacheKey; import org.apache.tajo.rpc.NettyUtils; -import org.apache.tajo.storage.*; -import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder; import org.apache.tajo.storage.index.bst.BSTIndex; import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader; -import org.apache.tajo.unit.StorageUnit; import org.apache.tajo.util.TajoIdUtils; import java.io.*; import java.net.InetSocketAddress; -import java.net.URI; -import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.List; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; + +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; public class TajoPullServerService extends AbstractService { private static final Log LOG = LogFactory.getLog(TajoPullServerService.class); - public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache"; - public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true; - - public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes"; - public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024; - private int port; private ServerBootstrap selector; private final ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); @@ -99,7 +93,6 @@ public class TajoPullServerService extends AbstractService { private int sslFileBufferSize; private int maxUrlLength; - private ApplicationId appId; private FileSystem localFS; /** @@ -110,62 +103,14 @@ public class TajoPullServerService extends AbstractService { private int readaheadLength; private ReadaheadPool readaheadPool = ReadaheadPool.getInstance(); - public static final String PULLSERVER_SERVICEID = "tajo.pullserver"; - - private static final Map userRsrc = - new ConcurrentHashMap<>(); - private String userName; - - private static LoadingCache indexReaderCache = null; - private static int lowCacheHitCheckThreshold; - - public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY = - "tajo.pullserver.ssl.file.buffer.size"; - - public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024; + private LoadingCache indexReaderCache = null; + private int lowCacheHitCheckThreshold; private static final boolean STANDALONE; - private static final AtomicIntegerFieldUpdater SLOW_FILE_UPDATER; - private static final AtomicIntegerFieldUpdater REMAIN_FILE_UPDATER; - - public static final String CHUNK_LENGTH_HEADER_NAME = "c"; - - static class CacheKey { - private Path path; - private String queryId; - private String ebSeqId; - - public CacheKey(Path path, String queryId, String ebSeqId) { - this.path = path; - this.queryId = queryId; - this.ebSeqId = ebSeqId; - } - - @Override - public boolean equals(Object o) { - if (o instanceof CacheKey) { - CacheKey other = (CacheKey) o; - return Objects.equals(this.path, other.path) - && Objects.equals(this.queryId, other.queryId) - && Objects.equals(this.ebSeqId, other.ebSeqId); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hash(path, queryId, ebSeqId); - } - } - static { - /* AtomicIntegerFieldUpdater can save the memory usage instead of AtomicInteger instance */ - SLOW_FILE_UPDATER = AtomicIntegerFieldUpdater.newUpdater(ProcessingStatus.class, "numSlowFile"); - REMAIN_FILE_UPDATER = AtomicIntegerFieldUpdater.newUpdater(ProcessingStatus.class, "remainFiles"); - - String standalone = System.getenv("TAJO_PULLSERVER_STANDALONE"); - STANDALONE = !StringUtils.isEmpty(standalone) && standalone.equalsIgnoreCase("true"); + String standalone = System.getenv(PullServerConstants.PULLSERVER_STANDALONE_ENV_KEY); + STANDALONE = !StringUtils.isEmpty(standalone) && standalone.equalsIgnoreCase(Boolean.TRUE.toString()); } @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo") @@ -193,7 +138,7 @@ public void operationComplete(ChannelFuture future) throws Exception { final ShuffleMetrics metrics; TajoPullServerService(MetricsSystem ms) { - super("httpshuffle"); + super(PullServerConstants.PULLSERVER_SERVICE_NAME); metrics = ms.register(new ShuffleMetrics()); } @@ -202,58 +147,35 @@ public TajoPullServerService() { this(DefaultMetricsSystem.instance()); } - public void initApp(String user, ApplicationId appId, ByteBuffer secret) { - // TODO these bytes should be versioned - // TODO: Once SHuffle is out of NM, this can use MR APIs - this.appId = appId; - this.userName = user; - userRsrc.put(appId.toString(), user); - } - - public void stopApp(ApplicationId appId) { - userRsrc.remove(appId.toString()); - } - + // TODO change AbstractService to throw InterruptedException @Override - public void init(Configuration conf) { - try { - manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE, - DEFAULT_SHUFFLE_MANAGE_OS_CACHE); - - readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES, - DEFAULT_SHUFFLE_READAHEAD_BYTES); + public void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + TajoConf tajoConf = (TajoConf) conf; - int workerNum = conf.getInt("tajo.shuffle.rpc.server.worker-thread-num", - Runtime.getRuntime().availableProcessors() * 2); + manageOsCache = tajoConf.getBoolean(PullServerConstants.SHUFFLE_MANAGE_OS_CACHE, + PullServerConstants.DEFAULT_SHUFFLE_MANAGE_OS_CACHE); - selector = NettyUtils.createServerBootstrap("TajoPullServerService", workerNum) - .option(ChannelOption.TCP_NODELAY, true) - .childOption(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) - .childOption(ChannelOption.TCP_NODELAY, true); + readaheadLength = tajoConf.getInt(PullServerConstants.SHUFFLE_READAHEAD_BYTES, + PullServerConstants.DEFAULT_SHUFFLE_READAHEAD_BYTES); - localFS = new LocalFileSystem(); + int workerNum = tajoConf.getIntVar(ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM); - maxUrlLength = conf.getInt(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH.name(), - ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH.defaultIntVal); + selector = NettyUtils.createServerBootstrap("TajoPullServerService", workerNum) + .option(ChannelOption.TCP_NODELAY, true) + .childOption(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) + .childOption(ChannelOption.TCP_NODELAY, true); - conf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname - , conf.getInt(TajoConf.ConfVars.PULLSERVER_PORT.varname, TajoConf.ConfVars.PULLSERVER_PORT.defaultIntVal)); - super.init(conf); - LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength); - } catch (Throwable t) { - LOG.error(t, t); - } - } + localFS = new LocalFileSystem(); - // TODO change AbstractService to throw InterruptedException - @Override - public void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } + maxUrlLength = tajoConf.getIntVar(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH); +// conf.setInt(ConfVars.PULLSERVER_PORT.varname, +// conf.getInt(ConfVars.PULLSERVER_PORT.varname, ConfVars.PULLSERVER_PORT.defaultIntVal)); + LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength); ServerBootstrap bootstrap = selector.clone(); - TajoConf tajoConf = (TajoConf)conf; try { channelInitializer = new HttpChannelInitializer(tajoConf); } catch (Exception ex) { @@ -263,7 +185,7 @@ public void serviceInit(Configuration conf) throws Exception { .channel(NioServerSocketChannel.class); port = tajoConf.getIntVar(ConfVars.PULLSERVER_PORT); - ChannelFuture future = bootstrap.bind(new InetSocketAddress(port)) + ChannelFuture future = bootstrap.bind(port) .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE) .syncUninterruptibly(); @@ -272,8 +194,8 @@ public void serviceInit(Configuration conf) throws Exception { tajoConf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port)); LOG.info(getName() + " listening on port " + port); - sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, - DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); + sslFileBufferSize = conf.getInt(PullServerConstants.SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, + PullServerConstants.DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); int cacheSize = tajoConf.getIntVar(ConfVars.PULLSERVER_CACHE_SIZE); int cacheTimeout = tajoConf.getIntVar(ConfVars.PULLSERVER_CACHE_TIMEOUT); @@ -283,10 +205,10 @@ public void serviceInit(Configuration conf) throws Exception { .expireAfterWrite(cacheTimeout, TimeUnit.MINUTES) .removalListener(removalListener) .build( - new CacheLoader() { + new CacheLoader() { @Override - public BSTIndexReader load(CacheKey key) throws Exception { - return new BSTIndex(tajoConf).getIndexReader(new Path(key.path, "index")); + public BSTIndexReader load(IndexCacheKey key) throws Exception { + return new BSTIndex(tajoConf).getIndexReader(new Path(key.getPath(), "index")); } } ); @@ -353,29 +275,31 @@ public int getPort() { } @Override - public void stop() { - try { - accepted.close(); - if (selector != null) { - if (selector.group() != null) { - selector.group().shutdownGracefully(); - } - if (selector.childGroup() != null) { - selector.childGroup().shutdownGracefully(); - } + public void serviceStop() throws Exception { + accepted.close(); + if (selector != null) { + if (selector.group() != null) { + selector.group().shutdownGracefully(); } - - if (channelInitializer != null) { - channelInitializer.destroy(); + if (selector.childGroup() != null) { + selector.childGroup().shutdownGracefully(); } + } - localFS.close(); - indexReaderCache.invalidateAll(); - } catch (Throwable t) { - LOG.error(t, t); - } finally { - super.stop(); + if (channelInitializer != null) { + channelInitializer.destroy(); } + + localFS.close(); + indexReaderCache.invalidateAll(); + + super.serviceStop(); + } + + public List getFileChunks(TajoConf conf, LocalDirAllocator lDirAlloc, PullServerParams params) + throws IOException, ExecutionException { + return PullServerUtil.getFileChunks(conf, lDirAlloc, localFS, params, indexReaderCache, + lowCacheHitCheckThreshold); } class HttpChannelInitializer extends ChannelInitializer { @@ -385,8 +309,7 @@ class HttpChannelInitializer extends ChannelInitializer { public HttpChannelInitializer(TajoConf conf) throws Exception { PullServer = new PullServer(conf); - if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname, - ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) { + if (conf.getBoolVar(ConfVars.SHUFFLE_SSL_ENABLED_KEY)) { sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); sslFactory.init(); } @@ -417,72 +340,13 @@ protected void initChannel(SocketChannel channel) throws Exception { } } - - Map processingStatusMap = new ConcurrentHashMap<>(); - - public void completeFileChunk(FileRegion filePart, - String requestUri, - long startTime) { - ProcessingStatus status = processingStatusMap.get(requestUri); - if (status != null) { - status.decrementRemainFiles(filePart, startTime); - } - } - - class ProcessingStatus { - String requestUri; - int numFiles; - long startTime; - long makeFileListTime; - long minTime = Long.MAX_VALUE; - long maxTime; - volatile int numSlowFile; - volatile int remainFiles; - - public ProcessingStatus(String requestUri) { - this.requestUri = requestUri; - this.startTime = System.currentTimeMillis(); - } - - public void setNumFiles(int numFiles) { - this.numFiles = numFiles; - this.remainFiles = numFiles; - } - - public void decrementRemainFiles(FileRegion filePart, long fileStartTime) { - long fileSendTime = System.currentTimeMillis() - fileStartTime; - - if (fileSendTime > maxTime) { - maxTime = fileSendTime; - } - if (fileSendTime < minTime) { - minTime = fileSendTime; - } - - if (fileSendTime > 20 * 1000) { - LOG.warn("Sending data takes too long. " + fileSendTime + "ms elapsed, " + - "length:" + (filePart.count() - filePart.position()) + ", URI:" + requestUri); - SLOW_FILE_UPDATER.compareAndSet(this, numSlowFile, numSlowFile + 1); - } - - REMAIN_FILE_UPDATER.compareAndSet(this, remainFiles, remainFiles - 1); - if (REMAIN_FILE_UPDATER.get(this) <= 0) { - processingStatusMap.remove(requestUri); - if(LOG.isDebugEnabled()) { - LOG.debug("PullServer processing status: totalTime=" + (System.currentTimeMillis() - startTime) + " ms, " - + "makeFileListTime=" + makeFileListTime + " ms, minTime=" + minTime + " ms, maxTime=" + maxTime + " ms, " - + "numFiles=" + numFiles + ", numSlowFile=" + numSlowFile); - } - } - } - } - @ChannelHandler.Sharable class PullServer extends SimpleChannelInboundHandler { private final TajoConf conf; private final LocalDirAllocator lDirAlloc = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); + private final Gson gson = new Gson(); public PullServer(TajoConf conf) throws IOException { this.conf = conf; @@ -512,7 +376,7 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) } if (request.getMethod() == HttpMethod.DELETE) { - HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NO_CONTENT); + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, HttpResponseStatus.NO_CONTENT); ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); clearIndexCache(request.getUri()); @@ -523,110 +387,123 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) } // Parsing the URL into key-values - Map> params = null; try { - params = decodeParams(request.getUri()); + final PullServerParams params = new PullServerParams(request.getUri()); + if (PullServerUtil.isChunkRequest(params.requestType())) { + handleChunkRequest(ctx, request, params); + } else { + handleMetaRequest(ctx, request, params); + } } catch (Throwable e) { - LOG.error("Failed to decode uri " + request.getUri()); + LOG.error("Failed to handle request " + request.getUri()); sendError(ctx, e.getMessage(), HttpResponseStatus.BAD_REQUEST); return; } + } - ProcessingStatus processingStatus = new ProcessingStatus(request.getUri()); - processingStatusMap.put(request.getUri(), processingStatus); - - String partId = params.get("p").get(0); - String queryId = params.get("qid").get(0); - String shuffleType = params.get("type").get(0); - String sid = params.get("sid").get(0); - - final List taskIdList = params.get("ta"); - final List offsetList = params.get("offset"); - final List lengthList = params.get("length"); - - long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; - long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; - - List taskIds = splitMaps(taskIdList); - - Path queryBaseDir = getBaseOutputDir(queryId, sid); - - if (LOG.isDebugEnabled()) { - LOG.debug("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId - + ", taskIds=" + taskIdList); - - // the working dir of tajo worker for each query - LOG.debug("PullServer baseDir: " + conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname) + "/" + queryBaseDir); + /** + * Upon a request from TajoWorker, this method clears index cache for fetching data of an execution block. + * It is called whenever an execution block is completed. + * + * @param uri query URI which indicates the execution block id + * @throws IOException + * @throws InvalidURLException + */ + public void clearIndexCache(String uri) + throws IOException, InvalidURLException { + // Simply parse the given uri + String[] tokens = uri.split("="); + if (tokens.length != 2 || !tokens[0].equals("ebid")) { + throw new IllegalArgumentException("invalid params: " + uri); } - - final List chunks = Lists.newArrayList(); - - // if a stage requires a range shuffle - if (shuffleType.equals("r")) { - final String startKey = params.get("start").get(0); - final String endKey = params.get("end").get(0); - final boolean last = params.get("final") != null; - - long before = System.currentTimeMillis(); - for (String eachTaskId : taskIds) { - Path outputPath = StorageUtil.concatPath(queryBaseDir, eachTaskId, "output"); - if (!lDirAlloc.ifExists(outputPath.toString(), conf)) { - LOG.warn(outputPath + "does not exist."); - continue; - } - Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(outputPath.toString(), conf)); - - FileChunk chunk; - try { - chunk = getFileChunks(queryId, sid, path, startKey, endKey, last); - } catch (Throwable t) { - LOG.error("ERROR Request: " + request.getUri(), t); - sendError(ctx, "Cannot get file chunks to be sent", HttpResponseStatus.BAD_REQUEST); - return; + ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(tokens[1]); + String queryId = ebId.getQueryId().toString(); + String ebSeqId = Integer.toString(ebId.getId()); + List removed = new ArrayList<>(); + synchronized (indexReaderCache) { + for (Entry e : indexReaderCache.asMap().entrySet()) { + IndexCacheKey key = e.getKey(); + if (key.getQueryId().equals(queryId) && key.getEbSeqId().equals(ebSeqId)) { + e.getValue().forceClose(); + removed.add(e.getKey()); } - if (chunk != null) { - chunks.add(chunk); + } + indexReaderCache.invalidateAll(removed); + } + removed.clear(); + synchronized (waitForRemove) { + for (Entry e : waitForRemove.entrySet()) { + IndexCacheKey key = e.getKey(); + if (key.getQueryId().equals(queryId) && key.getEbSeqId().equals(ebSeqId)) { + e.getValue().forceClose(); + removed.add(e.getKey()); } } - long after = System.currentTimeMillis(); - LOG.info("Index lookup time: " + (after - before) + " ms"); - - // if a stage requires a hash shuffle or a scattered hash shuffle - } else if (shuffleType.equals("h") || shuffleType.equals("s")) { - int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), conf); - Path partPath = StorageUtil.concatPath(queryBaseDir, "hash-shuffle", String.valueOf(partParentId), partId); - if (!lDirAlloc.ifExists(partPath.toString(), conf)) { - LOG.warn("Partition shuffle file not exists: " + partPath); - sendError(ctx, HttpResponseStatus.NO_CONTENT); - return; + for (IndexCacheKey eachKey : removed) { + waitForRemove.remove(eachKey); } + } + } + + private void handleMetaRequest(ChannelHandlerContext ctx, FullHttpRequest request, final PullServerParams params) + throws IOException, ExecutionException { + final List jsonMetas; + try { + jsonMetas = PullServerUtil.getJsonMeta(conf, lDirAlloc, localFS, params, gson, indexReaderCache, + lowCacheHitCheckThreshold); +// } catch (Throwable t) { +// // TODO: error handling +// LOG.error("Cannot find the file chunk meta for " + request.getUri(), t); +// sendError(ctx, "Cannot get file chunks to be sent", HttpResponseStatus.BAD_REQUEST); +// return; +// } + } catch (FileNotFoundException e) { + sendError(ctx, e.getMessage(), HttpResponseStatus.NO_CONTENT); + return; + } catch (IOException | IllegalArgumentException e) { // IOException, EOFException, IllegalArgumentException + sendError(ctx, e.getMessage(), HttpResponseStatus.BAD_REQUEST); + return; + } catch (ExecutionException e) { + // There are some problems in index cache + throw new TajoInternalError(e.getCause()); + } - Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(partPath.toString(), conf)); + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, HttpResponseStatus.OK, + Unpooled.copiedBuffer(gson.toJson(jsonMetas), CharsetUtil.UTF_8)); + response.headers().set(Names.CONTENT_TYPE, "application/json; charset=UTF-8"); + HttpHeaders.setContentLength(response, response.content().readableBytes()); + if (HttpHeaders.isKeepAlive(request)) { + response.headers().set(Names.CONNECTION, Values.KEEP_ALIVE); + } + ChannelFuture writeFuture = ctx.writeAndFlush(response); - File file = new File(path.toUri()); - long startPos = (offset >= 0 && length >= 0) ? offset : 0; - long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + // Decide whether to close the connection or not. + if (!HttpHeaders.isKeepAlive(request)) { + // Close the connection when the whole content is written out. + writeFuture.addListener(ChannelFutureListener.CLOSE); + } + } - if (startPos >= file.length()) { - String errorMessage = "Start pos[" + startPos + "] great than file length [" + file.length() + "]"; - LOG.error(errorMessage); - sendError(ctx, errorMessage, HttpResponseStatus.BAD_REQUEST); - return; - } - if (LOG.isDebugEnabled()) { - LOG.debug("RequestURL: " + request.getUri() + ", fileLen=" + file.length()); - } - FileChunk chunk = new FileChunk(file, startPos, readLen); - chunks.add(chunk); - } else { - LOG.error("Unknown shuffle type: " + shuffleType); - sendError(ctx, "Unknown shuffle type:" + shuffleType, HttpResponseStatus.BAD_REQUEST); + private void handleChunkRequest(ChannelHandlerContext ctx, FullHttpRequest request, final PullServerParams params) + throws IOException { + final List chunks; + try { + chunks = PullServerUtil.getFileChunks(conf, lDirAlloc, localFS, params, indexReaderCache, + lowCacheHitCheckThreshold); + } catch (FileNotFoundException e) { + sendError(ctx, e.getMessage(), HttpResponseStatus.NO_CONTENT); + return; + } catch (IOException | IllegalArgumentException e) { // IOException, EOFException, IllegalArgumentException + sendError(ctx, e.getMessage(), HttpResponseStatus.BAD_REQUEST); return; + } catch (ExecutionException e) { + // There are some problems in index cache + throw new TajoInternalError(e.getCause()); } // Write the content. if (chunks.size() == 0) { - HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NO_CONTENT); + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, HttpResponseStatus.NO_CONTENT); if (!HttpHeaders.isKeepAlive(request)) { ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); @@ -637,7 +514,7 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) } else { FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]); ChannelFuture writeFuture = null; - HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, HttpResponseStatus.OK); long totalSize = 0; StringBuilder sb = new StringBuilder(); for (FileChunk chunk : file) { @@ -645,7 +522,7 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) sb.append(Long.toString(chunk.length())).append(","); } sb.deleteCharAt(sb.length() - 1); - HttpHeaders.addHeader(response, CHUNK_LENGTH_HEADER_NAME, sb.toString()); + HttpHeaders.addHeader(response, PullServerConstants.CHUNK_LENGTH_HEADER_NAME, sb.toString()); HttpHeaders.setContentLength(response, totalSize); if (HttpHeaders.isKeepAlive(request)) { @@ -655,7 +532,7 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) writeFuture = ctx.write(response); for (FileChunk chunk : file) { - writeFuture = sendFile(ctx, chunk, request.getUri()); + writeFuture = sendFile(ctx, chunk); if (writeFuture == null) { sendError(ctx, HttpResponseStatus.NOT_FOUND); return; @@ -676,53 +553,8 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) } } - /** - * Upon a request from TajoWorker, this method clears index cache for fetching data of an execution block. - * It is called whenever an execution block is completed. - * - * @param uri query URI which indicates the execution block id - * @throws IOException - * @throws InvalidURLException - */ - private void clearIndexCache(String uri) throws IOException, InvalidURLException { - // Simply parse the given uri - String[] tokens = uri.split("="); - if (tokens.length != 2 || !tokens[0].equals("ebid")) { - throw new IllegalArgumentException("invalid params: " + uri); - } - ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(tokens[1]); - String queryId = ebId.getQueryId().toString(); - String ebSeqId = Integer.toString(ebId.getId()); - List removed = new ArrayList<>(); - synchronized (indexReaderCache) { - for (Entry e : indexReaderCache.asMap().entrySet()) { - CacheKey key = e.getKey(); - if (key.queryId.equals(queryId) && key.ebSeqId.equals(ebSeqId)) { - e.getValue().forceClose(); - removed.add(e.getKey()); - } - } - indexReaderCache.invalidateAll(removed); - } - removed.clear(); - synchronized (waitForRemove) { - for (Entry e : waitForRemove.entrySet()) { - CacheKey key = e.getKey(); - if (key.queryId.equals(queryId) && key.ebSeqId.equals(ebSeqId)) { - e.getValue().forceClose(); - removed.add(e.getKey()); - } - } - for (CacheKey eachKey : removed) { - waitForRemove.remove(eachKey); - } - } - } - private ChannelFuture sendFile(ChannelHandlerContext ctx, - FileChunk file, - String requestUri) throws IOException { - long startTime = System.currentTimeMillis(); + FileChunk file) throws IOException { RandomAccessFile spill = null; ChannelFuture writeFuture; try { @@ -732,7 +564,7 @@ private ChannelFuture sendFile(ChannelHandlerContext ctx, file.startOffset(), file.length(), manageOsCache, readaheadLength, readaheadPool, file.getFile().getAbsolutePath()); writeFuture = ctx.write(filePart); - writeFuture.addListener(new FileCloseListener(filePart, requestUri, startTime, TajoPullServerService.this)); + writeFuture.addListener(new FileCloseListener(filePart)); } else { // HTTPS cannot be done with zero copy. final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill, @@ -763,9 +595,10 @@ private void sendError(ChannelHandlerContext ctx, private void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - FullHttpResponse response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, status, - Unpooled.copiedBuffer(message, CharsetUtil.UTF_8)); + ByteBuf content = Unpooled.copiedBuffer(message, CharsetUtil.UTF_8); + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status, content); response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=UTF-8"); + HttpHeaders.setContentLength(response, content.writerIndex()); // Close the connection as soon as the error message is sent. ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); @@ -782,12 +615,12 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) } // Temporal space to wait for the completion of all index lookup operations - private static final ConcurrentHashMap waitForRemove = new ConcurrentHashMap<>(); + private final ConcurrentHashMap waitForRemove = new ConcurrentHashMap<>(); // RemovalListener is triggered when an item is removed from the index reader cache. // It closes index readers when they are not used anymore. // If they are still being used, they are moved to waitForRemove map to wait for other operations' completion. - private static final RemovalListener removalListener = (removal) -> { + private final RemovalListener removalListener = (removal) -> { BSTIndexReader reader = removal.getValue(); if (reader.getReferenceNum() == 0) { try { @@ -800,180 +633,4 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) waitForRemove.put(removal.getKey(), reader); } }; - - public static FileChunk getFileChunks(String queryId, - String ebSeqId, - Path outDir, - String startKey, - String endKey, - boolean last) throws IOException, ExecutionException { - - BSTIndexReader idxReader = indexReaderCache.get(new CacheKey(outDir, queryId, ebSeqId)); - idxReader.retain(); - - File data; - long startOffset; - long endOffset; - try { - if (LOG.isDebugEnabled()) { - if (indexReaderCache.size() > lowCacheHitCheckThreshold && indexReaderCache.stats().hitRate() < 0.5) { - LOG.debug("Too low cache hit rate: " + indexReaderCache.stats()); - } - } - - Tuple indexedFirst = idxReader.getFirstKey(); - Tuple indexedLast = idxReader.getLastKey(); - - if (indexedFirst == null && indexedLast == null) { // if # of rows is zero - if (LOG.isDebugEnabled()) { - LOG.debug("There is no contents"); - } - return null; - } - - byte[] startBytes = Base64.decodeBase64(startKey); - byte[] endBytes = Base64.decodeBase64(endKey); - - - Tuple start; - Tuple end; - Schema keySchema = idxReader.getKeySchema(); - RowStoreDecoder decoder = RowStoreUtil.createDecoder(keySchema); - - try { - start = decoder.toTuple(startBytes); - } catch (Throwable t) { - throw new IllegalArgumentException("StartKey: " + startKey - + ", decoded byte size: " + startBytes.length, t); - } - - try { - end = decoder.toTuple(endBytes); - } catch (Throwable t) { - throw new IllegalArgumentException("EndKey: " + endKey - + ", decoded byte size: " + endBytes.length, t); - } - - data = new File(URI.create(outDir.toUri() + "/output")); - if (LOG.isDebugEnabled()) { - LOG.debug("GET Request for " + data.getAbsolutePath() + " (start=" + start + ", end=" + end + - (last ? ", last=true" : "") + ")"); - } - - TupleComparator comparator = idxReader.getComparator(); - - if (comparator.compare(end, indexedFirst) < 0 || - comparator.compare(indexedLast, start) < 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("Out of Scope (indexed data [" + indexedFirst + ", " + indexedLast + - "], but request start:" + start + ", end: " + end); - } - return null; - } - - try { - idxReader.init(); - startOffset = idxReader.find(start); - } catch (IOException ioe) { - LOG.error("State Dump (the requested range: " - + "[" + start + ", " + end + ")" + ", idx min: " - + idxReader.getFirstKey() + ", idx max: " - + idxReader.getLastKey()); - throw ioe; - } - try { - endOffset = idxReader.find(end); - if (endOffset == -1) { - endOffset = idxReader.find(end, true); - } - } catch (IOException ioe) { - LOG.error("State Dump (the requested range: " - + "[" + start + ", " + end + ")" + ", idx min: " - + idxReader.getFirstKey() + ", idx max: " - + idxReader.getLastKey()); - throw ioe; - } - - // if startOffset == -1 then case 2-1 or case 3 - if (startOffset == -1) { // this is a hack - // if case 2-1 or case 3 - try { - startOffset = idxReader.find(start, true); - } catch (IOException ioe) { - LOG.error("State Dump (the requested range: " - + "[" + start + ", " + end + ")" + ", idx min: " - + idxReader.getFirstKey() + ", idx max: " - + idxReader.getLastKey()); - throw ioe; - } - } - - if (startOffset == -1) { - throw new IllegalStateException("startOffset " + startOffset + " is negative \n" + - "State Dump (the requested range: " - + "[" + start + ", " + end + ")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: " - + idxReader.getLastKey()); - } - - // if greater than indexed values - if (last || (endOffset == -1 - && comparator.compare(idxReader.getLastKey(), end) < 0)) { - endOffset = data.length(); - } - } finally { - idxReader.release(); - } - - FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset); - - if (LOG.isDebugEnabled()) LOG.debug("Retrieve File Chunk: " + chunk); - return chunk; - } - - public static List splitMaps(List mapq) { - if (null == mapq) { - return null; - } - final List ret = new ArrayList<>(); - for (String s : mapq) { - Collections.addAll(ret, s.split(",")); - } - return ret; - } - - public static Map> decodeParams(String uri) { - final Map> params = new QueryStringDecoder(uri).parameters(); - final List types = params.get("type"); - final List qids = params.get("qid"); - final List ebIds = params.get("sid"); - final List partIds = params.get("p"); - - if (types == null || ebIds == null || qids == null || partIds == null) { - throw new IllegalArgumentException("invalid params. required :" + params); - } - - if (qids.size() != 1 && types.size() != 1 || ebIds.size() != 1) { - throw new IllegalArgumentException("invalid params. required :" + params); - } - - return params; - } - - public static Path getBaseOutputDir(String queryId, String executionBlockSequenceId) { - Path workDir = - StorageUtil.concatPath( - queryId, - "output", - executionBlockSequenceId); - return workDir; - } - - public static Path getBaseInputDir(String queryId, String executionBlockId) { - Path workDir = - StorageUtil.concatPath( - queryId, - "in", - executionBlockId); - return workDir; - } } diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java index 67cff217c3..c5f6a6aded 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java @@ -36,7 +36,7 @@ public class FileChunk { */ private String ebId; - public FileChunk(File file, long startOffset, long length) throws FileNotFoundException { + public FileChunk(File file, long startOffset, long length) { this.file = file; this.startOffset = startOffset; this.length = length; @@ -76,6 +76,6 @@ public void setEbId(String newVal) { public String toString() { return " (start=" + startOffset() + ", length=" + length + ", fromRemote=" + fromRemote + ", ebId=" + ebId + ") " - + file.getAbsolutePath(); + + file.getAbsolutePath(); } } diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunkMeta.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunkMeta.java new file mode 100644 index 0000000000..3f6b3ebea4 --- /dev/null +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunkMeta.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.pullserver.retriever; + +public class FileChunkMeta { + private final long startOffset; + private final long length; + private final String ebId; + private final String taskId; + + public FileChunkMeta(long startOffset, long length, String ebId, String taskId) { + this.startOffset = startOffset; + this.length = length; + this.ebId = ebId; + this.taskId = taskId; + } + + public String getTaskId() { + return taskId; + } + + public long getStartOffset() { + return startOffset; + } + + public long getLength() { + return length; + } + + public String getEbId() { + return ebId; + } + + public String toString() { + return "ebId: " + ebId + ", taskId: " + taskId + " (" + startOffset + ", " + length + ")"; + } +} diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/IndexCacheKey.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/IndexCacheKey.java new file mode 100644 index 0000000000..2a71c65150 --- /dev/null +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/IndexCacheKey.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.pullserver.retriever; + +import org.apache.hadoop.fs.Path; + +import java.util.Objects; + +public class IndexCacheKey { + private Path path; + private String queryId; + private String ebSeqId; + + public IndexCacheKey(Path path, String queryId, String ebSeqId) { + this.path = path; + this.queryId = queryId; + this.ebSeqId = ebSeqId; + } + + public Path getPath() { + return path; + } + + public String getQueryId() { + return queryId; + } + + public String getEbSeqId() { + return ebSeqId; + } + + @Override + public boolean equals(Object o) { + if (o instanceof IndexCacheKey) { + IndexCacheKey other = (IndexCacheKey) o; + return Objects.equals(this.path, other.path) + && Objects.equals(this.queryId, other.queryId) + && Objects.equals(this.ebSeqId, other.ebSeqId); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(path, queryId, ebSeqId); + } +} diff --git a/tajo-yarn/pom.xml b/tajo-yarn/pom.xml new file mode 100644 index 0000000000..12061bfc7d --- /dev/null +++ b/tajo-yarn/pom.xml @@ -0,0 +1,185 @@ + + + + + + tajo-project + org.apache.tajo + 0.12.0-SNAPSHOT + ../tajo-project + + 4.0.0 + tajo-yarn + jar + Tajo Yarn + + UTF-8 + UTF-8 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.rat + apache-rat-plugin + + + verify + + check + + + + + + org.apache.maven.plugins + maven-surefire-report-plugin + + + maven-assembly-plugin + 2.4.1 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + + + + org.apache.hadoop + hadoop-mapreduce-client-shuffle + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-common + provided + + + org.apache.tajo + tajo-common + + + org.apache.tajo + tajo-storage-common + + + org.apache.tajo + tajo-storage-hdfs + + + org.apache.tajo + tajo-pullserver + + + + + docs + + false + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + module-javadocs + package + + jar + + + ${project.build.directory} + + + + + + + + + src + + false + + + + + org.apache.maven.plugins + maven-source-plugin + + + + tajo-java-sources + package + + jar-no-fork + + + + + + + + + + + + + org.apache.maven.plugins + maven-project-info-reports-plugin + 2.4 + + false + + + + org.apache.maven.plugins + maven-surefire-report-plugin + + + + + \ No newline at end of file diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedChunkedFile.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedChunkedFile.java new file mode 100644 index 0000000000..3c0a76f86f --- /dev/null +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedChunkedFile.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.yarn; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.ReadaheadPool; +import org.apache.hadoop.io.nativeio.NativeIO; +import org.apache.tajo.pullserver.PullServerUtil; +import org.jboss.netty.handler.stream.ChunkedFile; + +import java.io.FileDescriptor; +import java.io.IOException; +import java.io.RandomAccessFile; + +public class FadvisedChunkedFile extends ChunkedFile { + + private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class); + + private final boolean manageOsCache; + private final int readaheadLength; + private final ReadaheadPool readaheadPool; + private final FileDescriptor fd; + private final String identifier; + + private ReadaheadPool.ReadaheadRequest readaheadRequest; + + public FadvisedChunkedFile(RandomAccessFile file, long position, long count, + int chunkSize, boolean manageOsCache, int readaheadLength, + ReadaheadPool readaheadPool, String identifier) throws IOException { + super(file, position, count, chunkSize); + this.manageOsCache = manageOsCache; + this.readaheadLength = readaheadLength; + this.readaheadPool = readaheadPool; + this.fd = file.getFD(); + this.identifier = identifier; + } + + @Override + public Object nextChunk() throws Exception { + if (PullServerUtil.isNativeIOPossible() && manageOsCache && readaheadPool != null) { + readaheadRequest = readaheadPool + .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength, + getEndOffset(), readaheadRequest); + } + return super.nextChunk(); + } + + @Override + public void close() throws Exception { + if (readaheadRequest != null) { + readaheadRequest.cancel(); + } + if (PullServerUtil.isNativeIOPossible() && manageOsCache && getEndOffset() - getStartOffset() > 0) { + try { + PullServerUtil.posixFadviseIfPossible(identifier, + fd, + getStartOffset(), getEndOffset() - getStartOffset(), + NativeIO.POSIX.POSIX_FADV_DONTNEED); + } catch (Throwable t) { + LOG.warn("Failed to manage OS cache for " + identifier, t); + } + } + super.close(); + } +} diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java new file mode 100644 index 0000000000..ab2414de67 --- /dev/null +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java @@ -0,0 +1,171 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.yarn; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.ReadaheadPool; +import org.apache.hadoop.io.nativeio.NativeIO; +import org.apache.tajo.pullserver.PullServerUtil; +import org.jboss.netty.channel.DefaultFileRegion; + +import java.io.FileDescriptor; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; + +public class FadvisedFileRegion extends DefaultFileRegion { + + private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class); + + private final boolean manageOsCache; + private final int readaheadLength; + private final ReadaheadPool readaheadPool; + private final FileDescriptor fd; + private final String identifier; + private final long count; + private final long position; + private final int shuffleBufferSize; + private final boolean shuffleTransferToAllowed; + private final FileChannel fileChannel; + + private ReadaheadPool.ReadaheadRequest readaheadRequest; + public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024; + + public FadvisedFileRegion(RandomAccessFile file, long position, long count, + boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool, + String identifier) throws IOException { + this(file, position, count, manageOsCache, readaheadLength, readaheadPool, + identifier, DEFAULT_SHUFFLE_BUFFER_SIZE, true); + } + + public FadvisedFileRegion(RandomAccessFile file, long position, long count, + boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool, + String identifier, int shuffleBufferSize, + boolean shuffleTransferToAllowed) throws IOException { + super(file.getChannel(), position, count); + this.manageOsCache = manageOsCache; + this.readaheadLength = readaheadLength; + this.readaheadPool = readaheadPool; + this.fd = file.getFD(); + this.identifier = identifier; + this.fileChannel = file.getChannel(); + this.count = count; + this.position = position; + this.shuffleBufferSize = shuffleBufferSize; + this.shuffleTransferToAllowed = shuffleTransferToAllowed; + } + + @Override + public long transferTo(WritableByteChannel target, long position) + throws IOException { + if (PullServerUtil.isNativeIOPossible() && manageOsCache && readaheadPool != null) { + readaheadRequest = readaheadPool.readaheadStream(identifier, fd, + getPosition() + position, readaheadLength, + getPosition() + getCount(), readaheadRequest); + } + + if(this.shuffleTransferToAllowed) { + return super.transferTo(target, position); + } else { + return customShuffleTransfer(target, position); + } + } + + /** + * This method transfers data using local buffer. It transfers data from + * a disk to a local buffer in memory, and then it transfers data from the + * buffer to the target. This is used only if transferTo is disallowed in + * the configuration file. super.TransferTo does not perform well on Windows + * due to a small IO request generated. customShuffleTransfer can control + * the size of the IO requests by changing the size of the intermediate + * buffer. + */ + @VisibleForTesting + long customShuffleTransfer(WritableByteChannel target, long position) + throws IOException { + long actualCount = this.count - position; + if (actualCount < 0 || position < 0) { + throw new IllegalArgumentException( + "position out of range: " + position + + " (expected: 0 - " + (this.count - 1) + ')'); + } + if (actualCount == 0) { + return 0L; + } + + long trans = actualCount; + int readSize; + ByteBuffer byteBuffer = ByteBuffer.allocate(this.shuffleBufferSize); + + while(trans > 0L && + (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) { + //adjust counters and buffer limit + if(readSize < trans) { + trans -= readSize; + position += readSize; + byteBuffer.flip(); + } else { + //We can read more than we need if the actualCount is not multiple + //of the byteBuffer size and file is big enough. In that case we cannot + //use flip method but we need to set buffer limit manually to trans. + byteBuffer.limit((int)trans); + byteBuffer.position(0); + position += trans; + trans = 0; + } + + //write data to the target + while(byteBuffer.hasRemaining()) { + target.write(byteBuffer); + } + + byteBuffer.clear(); + } + + return actualCount - trans; + } + + + @Override + public void releaseExternalResources() { + if (readaheadRequest != null) { + readaheadRequest.cancel(); + } + super.releaseExternalResources(); + } + + /** + * Call when the transfer completes successfully so we can advise the OS that + * we don't need the region to be cached anymore. + */ + public void transferSuccessful() { + if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0) { + try { + PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), + NativeIO.POSIX.POSIX_FADV_DONTNEED); + } catch (Throwable t) { + LOG.warn("Failed to manage OS cache for " + identifier, t); + } + } + } +} diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java new file mode 100644 index 0000000000..066cdc8336 --- /dev/null +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.yarn; + +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; + +public class FileCloseListener implements ChannelFutureListener { + + private FadvisedFileRegion filePart; + + public FileCloseListener(FadvisedFileRegion filePart) { + this.filePart = filePart; + } + + // TODO error handling; distinguish IO/connection failures, + // attribute to appropriate spill output + @Override + public void operationComplete(ChannelFuture future) { + if(future.isSuccess()){ + filePart.transferSuccessful(); + } + filePart.releaseExternalResources(); +// if (pullServerService != null) { +// pullServerService.completeFileChunk(filePart, requestUri, startTime); +// } + } +} diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java new file mode 100644 index 0000000000..fc86118c24 --- /dev/null +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java @@ -0,0 +1,607 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.yarn; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.cache.RemovalListener; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.gson.Gson; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.ReadaheadPool; +import org.apache.hadoop.metrics2.MetricsSystem; +import org.apache.hadoop.metrics2.annotation.Metric; +import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MutableCounterInt; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; +import org.apache.hadoop.metrics2.lib.MutableGaugeInt; +import org.apache.hadoop.security.ssl.SSLFactory; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; +import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; +import org.apache.hadoop.yarn.server.api.AuxiliaryService; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.exception.InvalidURLException; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.pullserver.PullServerConstants; +import org.apache.tajo.pullserver.PullServerUtil; +import org.apache.tajo.pullserver.PullServerUtil.PullServerParams; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.pullserver.retriever.IndexCacheKey; +import org.apache.tajo.storage.index.bst.BSTIndex; +import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader; +import org.apache.tajo.util.SizeOf; +import org.apache.tajo.util.TajoIdUtils; +import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.channel.*; +import org.jboss.netty.channel.group.ChannelGroup; +import org.jboss.netty.channel.group.DefaultChannelGroup; +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.jboss.netty.handler.codec.http.*; +import org.jboss.netty.handler.codec.http.HttpHeaders.Names; +import org.jboss.netty.handler.codec.http.HttpHeaders.Values; +import org.jboss.netty.handler.ssl.SslHandler; +import org.jboss.netty.handler.stream.ChunkedWriteHandler; +import org.jboss.netty.util.CharsetUtil; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.*; + +import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; + +public class TajoPullServerService extends AuxiliaryService { + + private static final Log LOG = LogFactory.getLog(TajoPullServerService.class); + + private final TajoConf tajoConf; + + private int port; + private ChannelFactory selector; + private final ChannelGroup accepted = new DefaultChannelGroup("Pull server group"); + private HttpChannelInitializer channelInitializer; + private int sslFileBufferSize; + private int maxUrlLength; + + private ApplicationId appId; + private FileSystem localFS; + + /** + * Should the shuffle use posix_fadvise calls to manage the OS cache during + * sendfile + */ + private boolean manageOsCache; + private int readaheadLength; + private ReadaheadPool readaheadPool = ReadaheadPool.getInstance(); + + private static final Map userRsrc = + new ConcurrentHashMap<>(); + private String userName; + + private LoadingCache indexReaderCache = null; + private int lowCacheHitCheckThreshold; + + @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo") + static class ShuffleMetrics implements ChannelFutureListener { + @Metric({"OutputBytes","PullServer output in bytes"}) + MutableCounterLong shuffleOutputBytes; + @Metric({"Failed","# of failed shuffle outputs"}) + MutableCounterInt shuffleOutputsFailed; + @Metric({"Succeeded","# of succeeded shuffle outputs"}) + MutableCounterInt shuffleOutputsOK; + @Metric({"Connections","# of current shuffle connections"}) + MutableGaugeInt shuffleConnections; + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + shuffleOutputsOK.incr(); + } else { + shuffleOutputsFailed.incr(); + } + shuffleConnections.decr(); + } + } + + final ShuffleMetrics metrics; + + TajoPullServerService(MetricsSystem ms) { + super(PullServerConstants.PULLSERVER_SERVICE_NAME); + metrics = ms.register(new ShuffleMetrics()); + tajoConf = new TajoConf(); + } + + @SuppressWarnings("UnusedDeclaration") + public TajoPullServerService() { + this(DefaultMetricsSystem.instance()); + } + + @Override + public void initializeApplication(ApplicationInitializationContext context) { + // TODO these bytes should be versioned + // TODO: Once SHuffle is out of NM, this can use MR APIs + String user = context.getUser(); + ApplicationId appId = context.getApplicationId(); + // ByteBuffer secret = context.getApplicationDataForService(); + userRsrc.put(appId.toString(), user); + } + + @Override + public void stopApplication(ApplicationTerminationContext context) { + userRsrc.remove(context.getApplicationId().toString()); + } + + // TODO change AbstractService to throw InterruptedException + @Override + public void serviceInit(Configuration conf) throws Exception { + tajoConf.addResource(conf); + + manageOsCache = tajoConf.getBoolean(PullServerConstants.SHUFFLE_MANAGE_OS_CACHE, + PullServerConstants.DEFAULT_SHUFFLE_MANAGE_OS_CACHE); + + readaheadLength = tajoConf.getInt(PullServerConstants.SHUFFLE_READAHEAD_BYTES, + PullServerConstants.DEFAULT_SHUFFLE_READAHEAD_BYTES); + + int workerNum = tajoConf.getIntVar(ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM); + + ThreadFactory bossFactory = new ThreadFactoryBuilder() + .setNameFormat("TajoPullServerService Netty Boss #%d") + .build(); + ThreadFactory workerFactory = new ThreadFactoryBuilder() + .setNameFormat("TajoPullServerService Netty Worker #%d") + .build(); + selector = new NioServerSocketChannelFactory( + Executors.newCachedThreadPool(bossFactory), + Executors.newCachedThreadPool(workerFactory), + workerNum); + + localFS = new LocalFileSystem(); + + maxUrlLength = tajoConf.getIntVar(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH); + + LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength); + + ServerBootstrap bootstrap = new ServerBootstrap(selector); + try { + channelInitializer = new HttpChannelInitializer(tajoConf); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + bootstrap.setPipelineFactory(channelInitializer); + + port = tajoConf.getIntVar(ConfVars.PULLSERVER_PORT); + Channel ch = bootstrap.bind(new InetSocketAddress(port)); + + accepted.add(ch); + port = ((InetSocketAddress)ch.getLocalAddress()).getPort(); + tajoConf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port)); + LOG.info(getName() + " listening on port " + port); + + sslFileBufferSize = tajoConf.getInt(PullServerConstants.SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, + PullServerConstants.DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); + + int cacheSize = tajoConf.getIntVar(ConfVars.PULLSERVER_CACHE_SIZE); + int cacheTimeout = tajoConf.getIntVar(ConfVars.PULLSERVER_CACHE_TIMEOUT); + + indexReaderCache = CacheBuilder.newBuilder() + .maximumSize(cacheSize) + .expireAfterWrite(cacheTimeout, TimeUnit.MINUTES) + .removalListener(removalListener) + .build( + new CacheLoader() { + @Override + public BSTIndexReader load(IndexCacheKey key) throws Exception { + return new BSTIndex(tajoConf).getIndexReader(new Path(key.getPath(), "index")); + } + } + ); + lowCacheHitCheckThreshold = (int) (cacheSize * 0.1f); + + super.serviceInit(tajoConf); + LOG.info("TajoPullServerService started: port=" + port); + } + + @Override + public void serviceStop() throws Exception { + // TODO: check this wait + accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS); + if (selector != null) { + ServerBootstrap bootstrap = new ServerBootstrap(selector); + bootstrap.releaseExternalResources(); + } + + if (channelInitializer != null) { + channelInitializer.destroy(); + } + + localFS.close(); + indexReaderCache.invalidateAll(); + + super.serviceStop(); + } + + @VisibleForTesting + public int getPort() { + return port; + } + + @Override + public ByteBuffer getMetaData() { + try { + return serializeMetaData(port); + } catch (IOException e) { + LOG.error("Error during getMeta", e); + // TODO add API to AuxiliaryServices to report failures + return null; + } + } + + /** + * Serialize the shuffle port into a ByteBuffer for use later on. + * @param port the port to be sent to the ApplciationMaster + * @return the serialized form of the port. + */ + public static ByteBuffer serializeMetaData(int port) throws IOException { + //TODO these bytes should be versioned + return ByteBuffer.allocate(SizeOf.SIZE_OF_INT).putInt(port); + } + + class HttpChannelInitializer implements ChannelPipelineFactory { + + final PullServer PullServer; + private SSLFactory sslFactory; + + public HttpChannelInitializer(TajoConf conf) throws Exception { + PullServer = new PullServer(conf); + if (conf.getBoolVar(ConfVars.SHUFFLE_SSL_ENABLED_KEY)) { + sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); + sslFactory.init(); + } + } + + public void destroy() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } + + @Override + public ChannelPipeline getPipeline() throws Exception { + ChannelPipeline pipeline = Channels.pipeline(); + if (sslFactory != null) { + pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); + } + int maxChunkSize = getConfig().getInt(ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE.varname, + ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE.defaultIntVal); + pipeline.addLast("codec", new HttpServerCodec(maxUrlLength, 8192, maxChunkSize)); + pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16)); + pipeline.addLast("chunking", new ChunkedWriteHandler()); + pipeline.addLast("shuffle", PullServer); + return pipeline; + // TODO factor security manager into pipeline + // TODO factor out encode/decode to permit binary shuffle + // TODO factor out decode of index to permit alt. models + } + } + + @ChannelHandler.Sharable + class PullServer extends SimpleChannelUpstreamHandler { + + private final TajoConf conf; + private final LocalDirAllocator lDirAlloc = + new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); + private final Gson gson = new Gson(); + + public PullServer(TajoConf conf) throws IOException { + this.conf = conf; + + // init local temporal dir + lDirAlloc.getAllLocalPathsToRead(".", conf); + } + + @Override + public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) throws Exception { + accepted.add(evt.getChannel()); + + if(LOG.isDebugEnabled()) { + LOG.debug(String.format("Current number of shuffle connections (%d)", accepted.size())); + } + super.channelOpen(ctx, evt); + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) + throws Exception { + + HttpRequest request = (HttpRequest) evt.getMessage(); + Channel ch = evt.getChannel(); + + if (request.getMethod() == HttpMethod.DELETE) { + HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NO_CONTENT); + ch.write(response).addListener(ChannelFutureListener.CLOSE); + + clearIndexCache(request.getUri()); + return; + } else if (request.getMethod() != HttpMethod.GET) { + sendError(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED); + return; + } + + // Parsing the URL into key-values + try { + final PullServerParams params = new PullServerParams(request.getUri()); + if (PullServerUtil.isChunkRequest(params.requestType())) { + handleChunkRequest(ctx, request, params); + } else { + handleMetaRequest(ctx, request, params); + } + } catch (Throwable e) { + LOG.error("Failed to handle request " + request.getUri()); + sendError(ctx, e.getMessage(), HttpResponseStatus.BAD_REQUEST); + return; + } + } + + /** + * Upon a request from TajoWorker, this method clears index cache for fetching data of an execution block. + * It is called whenever an execution block is completed. + * + * @param uri query URI which indicates the execution block id + * @throws IOException + * @throws InvalidURLException + */ + public void clearIndexCache(String uri) + throws IOException, InvalidURLException { + // Simply parse the given uri + String[] tokens = uri.split("="); + if (tokens.length != 2 || !tokens[0].equals("ebid")) { + throw new IllegalArgumentException("invalid params: " + uri); + } + ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(tokens[1]); + String queryId = ebId.getQueryId().toString(); + String ebSeqId = Integer.toString(ebId.getId()); + List removed = new ArrayList<>(); + synchronized (indexReaderCache) { + for (Entry e : indexReaderCache.asMap().entrySet()) { + IndexCacheKey key = e.getKey(); + if (key.getQueryId().equals(queryId) && key.getEbSeqId().equals(ebSeqId)) { + e.getValue().forceClose(); + removed.add(e.getKey()); + } + } + indexReaderCache.invalidateAll(removed); + } + removed.clear(); + synchronized (waitForRemove) { + for (Entry e : waitForRemove.entrySet()) { + IndexCacheKey key = e.getKey(); + if (key.getQueryId().equals(queryId) && key.getEbSeqId().equals(ebSeqId)) { + e.getValue().forceClose(); + removed.add(e.getKey()); + } + } + for (IndexCacheKey eachKey : removed) { + waitForRemove.remove(eachKey); + } + } + } + + private void handleMetaRequest(ChannelHandlerContext ctx, HttpRequest request, final PullServerParams params) + throws IOException, ExecutionException { + final List jsonMetas; + try { + jsonMetas = PullServerUtil.getJsonMeta(conf, lDirAlloc, localFS, params, gson, indexReaderCache, + lowCacheHitCheckThreshold); + } catch (FileNotFoundException e) { + sendError(ctx, e.getMessage(), HttpResponseStatus.NO_CONTENT); + return; + } catch (IOException | IllegalArgumentException e) { // IOException, EOFException, IllegalArgumentException + sendError(ctx, e.getMessage(), HttpResponseStatus.BAD_REQUEST); + return; + } catch (ExecutionException e) { + // There are some problems in index cache + throw new TajoInternalError(e.getCause()); + } + + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, HttpResponseStatus.OK); + response.setContent(ChannelBuffers.copiedBuffer(gson.toJson(jsonMetas), CharsetUtil.UTF_8)); + response.setHeader(Names.CONTENT_TYPE, "application/json; charset=UTF-8"); + HttpHeaders.setContentLength(response, response.getContent().readableBytes()); + if (HttpHeaders.isKeepAlive(request)) { + response.setHeader(Names.CONNECTION, Values.KEEP_ALIVE); + } + ChannelFuture writeFuture = ctx.getChannel().write(response); + + // Decide whether to close the connection or not. + if (!HttpHeaders.isKeepAlive(request)) { + // Close the connection when the whole content is written out. + writeFuture.addListener(ChannelFutureListener.CLOSE); + } + } + + private void handleChunkRequest(ChannelHandlerContext ctx, HttpRequest request, final PullServerParams params) + throws IOException { + final List chunks; + try { + chunks = PullServerUtil.getFileChunks(conf, lDirAlloc, localFS, params, indexReaderCache, + lowCacheHitCheckThreshold); + } catch (FileNotFoundException e) { + sendError(ctx, e.getMessage(), HttpResponseStatus.NO_CONTENT); + return; + } catch (IOException | IllegalArgumentException e) { // IOException, EOFException, IllegalArgumentException + sendError(ctx, e.getMessage(), HttpResponseStatus.BAD_REQUEST); + return; + } catch (ExecutionException e) { + // There are some problems in index cache + throw new TajoInternalError(e.getCause()); + } + + // Write the content. + final Channel ch = ctx.getChannel(); + if (chunks.size() == 0) { + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, HttpResponseStatus.NO_CONTENT); + + if (!HttpHeaders.isKeepAlive(request)) { + ch.write(response).addListener(ChannelFutureListener.CLOSE); + } else { + response.setHeader(Names.CONNECTION, Values.KEEP_ALIVE); + ch.write(response); + } + } else { + FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]); + ChannelFuture writeFuture = null; + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, HttpResponseStatus.OK); + long totalSize = 0; + StringBuilder sb = new StringBuilder(); + for (FileChunk chunk : file) { + totalSize += chunk.length(); + sb.append(Long.toString(chunk.length())).append(","); + } + sb.deleteCharAt(sb.length() - 1); + HttpHeaders.addHeader(response, PullServerConstants.CHUNK_LENGTH_HEADER_NAME, sb.toString()); + HttpHeaders.setContentLength(response, totalSize); + + if (HttpHeaders.isKeepAlive(request)) { + response.setHeader(Names.CONNECTION, Values.KEEP_ALIVE); + } + // Write the initial line and the header. + writeFuture = ch.write(response); + + for (FileChunk chunk : file) { + writeFuture = sendFile(ctx, chunk); + if (writeFuture == null) { + sendError(ctx, HttpResponseStatus.NOT_FOUND); + return; + } + } + + // Decide whether to close the connection or not. + if (!HttpHeaders.isKeepAlive(request)) { + // Close the connection when the whole content is written out. + writeFuture.addListener(ChannelFutureListener.CLOSE); + } + } + } + + private ChannelFuture sendFile(ChannelHandlerContext ctx, + FileChunk file) throws IOException { + Channel ch = ctx.getChannel(); + RandomAccessFile spill = null; + ChannelFuture writeFuture; + try { + spill = new RandomAccessFile(file.getFile(), "r"); + if (ctx.getPipeline().get(SslHandler.class) == null) { + final FadvisedFileRegion filePart = new FadvisedFileRegion(spill, + file.startOffset(), file.length(), manageOsCache, readaheadLength, + readaheadPool, file.getFile().getAbsolutePath()); + writeFuture = ch.write(filePart); + writeFuture.addListener(new FileCloseListener(filePart)); + } else { + // HTTPS cannot be done with zero copy. + final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill, + file.startOffset(), file.length(), sslFileBufferSize, + manageOsCache, readaheadLength, readaheadPool, + file.getFile().getAbsolutePath()); + writeFuture = ch.write(chunk); + } + } catch (FileNotFoundException e) { + LOG.fatal(file.getFile() + " not found"); + return null; + } catch (Throwable e) { + if (spill != null) { + //should close a opening file + spill.close(); + } + return null; + } + metrics.shuffleConnections.incr(); + metrics.shuffleOutputBytes.incr(file.length()); // optimistic + return writeFuture; + } + + private void sendError(ChannelHandlerContext ctx, + HttpResponseStatus status) { + sendError(ctx, "", status); + } + + private void sendError(ChannelHandlerContext ctx, String message, + HttpResponseStatus status) { + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + response.setHeader(Names.CONTENT_TYPE, "text/plain; charset=UTF-8"); + // Put shuffle version into http header + ChannelBuffer content = ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8); + response.setContent(content); + response.setHeader(Names.CONTENT_LENGTH, content.writerIndex()); + + // Close the connection as soon as the error message is sent. + ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) + throws Exception { + Channel ch = e.getChannel(); + Throwable cause = e.getCause(); + LOG.error(cause.getMessage(), cause); + if (ch.isOpen()) { + ch.close(); + } + } + } + + // Temporal space to wait for the completion of all index lookup operations + private final ConcurrentHashMap waitForRemove = new ConcurrentHashMap<>(); + + // RemovalListener is triggered when an item is removed from the index reader cache. + // It closes index readers when they are not used anymore. + // If they are still being used, they are moved to waitForRemove map to wait for other operations' completion. + private final RemovalListener removalListener = (removal) -> { + BSTIndexReader reader = removal.getValue(); + if (reader.getReferenceNum() == 0) { + try { + reader.close(); // tear down properly + } catch (IOException e) { + throw new RuntimeException(e); + } + waitForRemove.remove(removal.getKey()); + } else { + waitForRemove.put(removal.getKey(), reader); + } + }; +} From eda7d71fe18daf7596e34118bd14547fadd550ef Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Apr 2016 13:58:58 +0900 Subject: [PATCH 02/20] Fix test failure --- .../planner/physical/TestProgressExternalSortExec.java | 2 -- .../java/org/apache/tajo/master/TestRepartitioner.java | 10 +++++----- .../org/apache/tajo/pullserver/PullServerUtil.java | 4 ++++ 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java index 21a08bd4fa..e6d5abd184 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java @@ -172,13 +172,11 @@ private void testProgress(QueryContext queryContext) throws Exception { while ((tuple = exec.next()) != null) { if (cnt == 0) { initProgress = exec.getProgress(); - System.out.println(initProgress); assertTrue(initProgress > 0.5f && initProgress < 1.0f); } if (cnt == testDataStats.getNumRows() / 2) { float progress = exec.getProgress(); - System.out.println(progress); assertTrue(progress > initProgress); } curVal = tuple; diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java index a13a7502a9..abec6a0443 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java @@ -20,12 +20,13 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.QueryId; import org.apache.tajo.ResourceProtos.FetchProto; import org.apache.tajo.TestTajoIds; +import org.apache.tajo.pullserver.PullServerConstants; +import org.apache.tajo.pullserver.PullServerUtil.PullServerParams; import org.apache.tajo.querymaster.Repartitioner; import org.apache.tajo.querymaster.Task; import org.apache.tajo.querymaster.Task.IntermediateEntry; @@ -88,12 +89,11 @@ public void testCreateHashFetchURL() throws Exception { assertEquals(1, uris.size()); //In Hash Suffle, Fetcher return only one URI per partition. URI uri = uris.get(0); - final Map> params = - new QueryStringDecoder(uri).parameters(); + final PullServerParams params = new PullServerParams(uri); assertEquals(eachEntry.getKey().toString(), params.get("p").get(0)); - assertEquals("h", params.get("type").get(0)); - assertEquals("" + sid.getId(), params.get("sid").get(0)); + assertEquals(PullServerConstants.HASH_SHUFFLE_PARAM_STRING, params.shuffleType()); + assertEquals("" + sid.getId(), params.ebId()); } Map>> mergedHashEntries = diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java index f9f665923f..4b69db99f7 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java @@ -161,6 +161,10 @@ public static boolean isHashShuffle(String shuffleType) { public static class PullServerParams extends HashMap> { + public PullServerParams(URI uri) { + this(uri.toString()); + } + public PullServerParams(String uri) { super(new QueryStringDecoder(uri).parameters()); } From df06b4ef8b82dd4f04baed0cb9756b59c3e96206 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Apr 2016 14:12:54 +0900 Subject: [PATCH 03/20] Make compatible with hadoop 2.3.0 --- .../java/org/apache/tajo/yarn/TajoPullServerService.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java index fc86118c24..91ab7b5126 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java @@ -90,7 +90,7 @@ public class TajoPullServerService extends AuxiliaryService { private static final Log LOG = LogFactory.getLog(TajoPullServerService.class); - private final TajoConf tajoConf; + private TajoConf tajoConf; private int port; private ChannelFactory selector; @@ -144,7 +144,6 @@ public void operationComplete(ChannelFuture future) throws Exception { TajoPullServerService(MetricsSystem ms) { super(PullServerConstants.PULLSERVER_SERVICE_NAME); metrics = ms.register(new ShuffleMetrics()); - tajoConf = new TajoConf(); } @SuppressWarnings("UnusedDeclaration") @@ -170,7 +169,7 @@ public void stopApplication(ApplicationTerminationContext context) { // TODO change AbstractService to throw InterruptedException @Override public void serviceInit(Configuration conf) throws Exception { - tajoConf.addResource(conf); + tajoConf = new TajoConf(conf); manageOsCache = tajoConf.getBoolean(PullServerConstants.SHUFFLE_MANAGE_OS_CACHE, PullServerConstants.DEFAULT_SHUFFLE_MANAGE_OS_CACHE); From 43abd54efdc9ad64fca1b53d8d7f52128939639e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 Apr 2016 17:09:44 +0900 Subject: [PATCH 04/20] Code cleanup and add a comment --- .../worker/TestFetcherWithTajoPullServer.java | 7 ++-- .../org/apache/tajo/worker/LocalFetcher.java | 21 ++++++++--- .../org/apache/tajo/worker/RemoteFetcher.java | 2 +- .../org/apache/tajo/worker/TajoWorker.java | 5 --- .../java/org/apache/tajo/worker/TaskImpl.java | 35 ++----------------- .../tajo/pullserver/FileCloseListener.java | 3 -- .../pullserver/TajoPullServerService.java | 8 ----- .../apache/tajo/yarn/FileCloseListener.java | 3 -- 8 files changed, 22 insertions(+), 62 deletions(-) diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java index 9c364afbbc..8844fce6ab 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcherWithTajoPullServer.java @@ -30,10 +30,7 @@ import org.apache.tajo.TajoProtos; import org.apache.tajo.TajoProtos.FetcherState; import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.catalog.SchemaFactory; -import org.apache.tajo.catalog.SortSpec; +import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -213,7 +210,7 @@ public void testGetRangeShuffle() throws IOException { } final FSDataOutputStream stream = fs.create(inputPath, true); BSTIndex index = new BSTIndex(conf); - Schema schema = SchemaFactory.newV1(new Column[] {new Column("rnd", Type.TEXT)}); + Schema schema = SchemaBuilder.builder().addAll(new Column[] {new Column("rnd", Type.TEXT)}).build(); SortSpec[] sortSpecs = new SortSpec[] {new SortSpec(schema.getColumn(0))}; BSTIndexWriter writer = index.getIndexWriter(new Path(INPUT_DIR, indexPath), BSTIndex.TWO_LEVEL_INDEX, schema, new BaseTupleComparator(schema, sortSpecs), true); writer.init(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java index 85f397f16e..e362a38c4f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java @@ -62,11 +62,24 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +/** + * LocalFetcher retrieves locally stored data. Its behavior can be different according to the pull server is running + * externally or internally. + * + *
    + *
  • When an internal pull server is running, local fetchers can retrieve data directly.
  • + *
  • When an external pull server is running,
  • + *
      + *
    • If the shuffle type is hash, local fetchers can still retrieve data directly.
    • + *
    • If the shuffle type is range, local fetchers need to get meta information of data via HTTP. Once the meta + * information is retrieved, they can read data directly.
    • + *
    + *
+ */ public class LocalFetcher extends AbstractFetcher { private final static Log LOG = LogFactory.getLog(LocalFetcher.class); -// private final ExecutionBlockContext executionBlockContext; private final TajoPullServerService pullServerService; private final String host; @@ -157,10 +170,10 @@ public LocalFetcher(TajoConf conf, URI uri, ExecutionBlockContext executionBlock @Override public List get() throws IOException { - return pullServerService != null ? getDirect() : getFromFetchURI(); + return pullServerService != null ? getWithInternalPullServer() : getWithExternalPullServer(); } - private List getDirect() throws IOException { + private List getWithInternalPullServer() throws IOException { final List fileChunks = new ArrayList<>(); startTime = System.currentTimeMillis(); PullServerParams params = new PullServerParams(uri.toString()); @@ -176,7 +189,7 @@ private List getDirect() throws IOException { return fileChunks; } - private List getFromFetchURI() throws IOException { + private List getWithExternalPullServer() throws IOException { final PullServerParams params = new PullServerParams(uri.toString()); final Path queryBaseDir = PullServerUtil.getBaseOutputDir(params.queryId(), params.ebId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java index 0e846126ff..fa87b9bd9d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java @@ -50,7 +50,7 @@ import java.util.concurrent.TimeUnit; /** - * Fetcher fetches data from a given uri via HTTP protocol and stores them into + * RemoteFetcher fetches data from a given uri via HTTP protocol and stores them into * a specific file. It aims at asynchronous and efficient data transmit. */ public class RemoteFetcher extends AbstractFetcher { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index f78404a756..6296bb088c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -149,11 +149,6 @@ public void serviceInit(Configuration conf) throws Exception { addIfService(pullService); } -// if(!TajoPullServerService.isStandalone()) { -// pullService = new TajoPullServerService(); -// addIfService(pullService); -// } - this.taskManager = new TaskManager(dispatcher, workerContext, pullService); addService(taskManager); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java index 753b4abca1..920dfe5205 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -698,46 +698,15 @@ private List getFetchRunners(TaskAttemptContext ctx, if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { localStoreChunkCount++; runnerList.add(new LocalFetcher(systemConf, uri, executionBlockContext, f.getName())); - -// List localChunkCandidates = getLocalStoredFileChunk(uri, systemConf); -// -// for (FileChunk localChunk : localChunkCandidates) { -// // When a range request is out of range, storeChunk will be NULL. This case is normal state. -// // So, we should skip and don't need to create storeChunk. -// if (localChunk == null || localChunk.length() == 0) { -// continue; -// } -// -// if (localChunk.getFile() != null && localChunk.startOffset() > -1) { -// localChunk.setFromRemote(false); -// localStoreChunkCount++; -// } else { -// localChunk = new FileChunk(defaultStoreFile, 0, -1); -// localChunk.setFromRemote(true); -// } -// localChunk.setEbId(f.getName()); -// storeChunkList.add(localChunk); -// } - } else { + // If we decide that intermediate data should be really fetched from a remote host, storeChunk + // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it FileChunk remoteChunk = new FileChunk(defaultStoreFile, 0, -1); remoteChunk.setFromRemote(true); remoteChunk.setEbId(f.getName()); -// storeChunkList.add(remoteChunk); runnerList.add(new RemoteFetcher(systemConf, uri, remoteChunk)); i++; } - - // If we decide that intermediate data should be really fetched from a remote host, storeChunk - // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it -// for (FileChunk eachChunk : storeChunkList) { -// RemoteFetcher fetcher = new RemoteFetcher(systemConf, uri, eachChunk); -// runnerList.add(fetcher); -// i++; -// if (LOG.isDebugEnabled()) { -// LOG.debug("Create a new Fetcher with storeChunk:" + eachChunk.toString()); -// } -// } } } ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java index 670f406099..7f97542ca1 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java @@ -37,8 +37,5 @@ public void operationComplete(ChannelFuture future) { filePart.transferSuccessful(); } filePart.deallocate(); -// if (pullServerService != null) { -// pullServerService.completeFileChunk(filePart, requestUri, startTime); -// } } } diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index 770248b203..aa16f8718e 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -171,8 +171,6 @@ public void serviceInit(Configuration conf) throws Exception { localFS = new LocalFileSystem(); maxUrlLength = tajoConf.getIntVar(ConfVars.PULLSERVER_FETCH_URL_MAX_LENGTH); -// conf.setInt(ConfVars.PULLSERVER_PORT.varname, -// conf.getInt(ConfVars.PULLSERVER_PORT.varname, ConfVars.PULLSERVER_PORT.defaultIntVal)); LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength); ServerBootstrap bootstrap = selector.clone(); @@ -451,12 +449,6 @@ private void handleMetaRequest(ChannelHandlerContext ctx, FullHttpRequest reques try { jsonMetas = PullServerUtil.getJsonMeta(conf, lDirAlloc, localFS, params, gson, indexReaderCache, lowCacheHitCheckThreshold); -// } catch (Throwable t) { -// // TODO: error handling -// LOG.error("Cannot find the file chunk meta for " + request.getUri(), t); -// sendError(ctx, "Cannot get file chunks to be sent", HttpResponseStatus.BAD_REQUEST); -// return; -// } } catch (FileNotFoundException e) { sendError(ctx, e.getMessage(), HttpResponseStatus.NO_CONTENT); return; diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java index 066cdc8336..a85d0b9e86 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FileCloseListener.java @@ -37,8 +37,5 @@ public void operationComplete(ChannelFuture future) { filePart.transferSuccessful(); } filePart.releaseExternalResources(); -// if (pullServerService != null) { -// pullServerService.completeFileChunk(filePart, requestUri, startTime); -// } } } From f131502469d99c21a5116aa8f4779fe4d512a7b7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 Apr 2016 17:22:49 +0900 Subject: [PATCH 05/20] Add dist --- tajo-dist/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml index 652ab84204..7280e1f2fc 100644 --- a/tajo-dist/pom.xml +++ b/tajo-dist/pom.xml @@ -160,6 +160,9 @@ run mkdir -p share/jdbc-dist run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar + run mkdir -p share/yarn-dist + run cp -r $ROOT/tajo-yarn/target/tajo-yarn-${project.version}-jar-with-dependencies.jar ./share/yarn-dist/tajo-yarn-${project.version}.jar + run mkdir -p extlib echo From 0ca94ae87d7031c5cbad1322520c07a1e841b0eb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 Apr 2016 18:22:41 +0900 Subject: [PATCH 06/20] trigger ci --- CHANGES | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGES b/CHANGES index 829f9a3aa7..07f33c3f94 100644 --- a/CHANGES +++ b/CHANGES @@ -4,6 +4,8 @@ Release 0.12.0 - unreleased NEW FEATURES + TAJO-2122: PullServer as an Auxiliary service of Yarn. (jihoon) + TAJO-2109: Implement Radix sort. (jihoon) TAJO-1955: Add a feature to strip quotes from CSV file. (hyunsik) From 552376a80283edf8ac974d55291c4568ba17fc96 Mon Sep 17 00:00:00 2001 From: Hyunsik Choi Date: Fri, 22 Apr 2016 05:04:17 -0700 Subject: [PATCH 07/20] TAJO-2127: Use Type instead of DataType for EvalNode. Closes #1005 --- CHANGES | 2 + .../java/org/apache/tajo/DataTypeUtil.java | 87 +++++++------- .../java/org/apache/tajo/catalog/Column.java | 10 ++ .../apache/tajo/catalog/TypeConverter.java | 56 ++++++--- .../InvalidValueForCastException.java | 25 ++-- .../main/java/org/apache/tajo/type/Int1.java | 28 +++++ .../main/java/org/apache/tajo/type/Type.java | 51 ++++---- .../org/apache/tajo/schema/TestSchema.java | 16 +-- .../java/org/apache/tajo/type/TestType.java | 52 +++++---- .../apache/tajo/engine/eval/ExprTestBase.java | 10 -- .../apache/tajo/engine/eval/TestEvalTree.java | 55 ++------- .../engine/planner/TestLogicalPlanner.java | 62 ---------- .../engine/codegen/EvalCodeGenContext.java | 2 +- .../engine/codegen/EvalCodeGenerator.java | 31 +++-- .../codegen/ScalarFunctionBindingEmitter.java | 5 +- .../engine/codegen/TajoGeneratorAdapter.java | 109 +++++++++--------- .../engine/codegen/VariablesPreBuilder.java | 4 +- .../engine/planner/global/GlobalPlanner.java | 8 +- .../worker/ExecutionBlockSharedResource.java | 6 +- .../org/apache/tajo/plan/ExprAnnotator.java | 45 ++++---- .../org/apache/tajo/plan/LogicalPlanner.java | 2 +- .../apache/tajo/plan/NamedExprsManager.java | 6 +- .../org/apache/tajo/plan/TypeDeterminant.java | 16 +-- .../expr/AggregationFunctionCallEval.java | 9 +- .../tajo/plan/expr/BetweenPredicateEval.java | 9 +- .../org/apache/tajo/plan/expr/BinaryEval.java | 18 +-- .../apache/tajo/plan/expr/CaseWhenEval.java | 15 +-- .../org/apache/tajo/plan/expr/CastEval.java | 7 +- .../org/apache/tajo/plan/expr/ConstEval.java | 8 +- .../org/apache/tajo/plan/expr/EvalNode.java | 3 +- .../apache/tajo/plan/expr/EvalTreeUtil.java | 2 +- .../org/apache/tajo/plan/expr/FieldEval.java | 11 +- .../apache/tajo/plan/expr/FunctionEval.java | 8 +- .../org/apache/tajo/plan/expr/InEval.java | 7 +- .../org/apache/tajo/plan/expr/IsNullEval.java | 13 +-- .../org/apache/tajo/plan/expr/NotEval.java | 10 +- .../tajo/plan/expr/PartialBinaryExpr.java | 4 +- .../plan/expr/PatternMatchPredicateEval.java | 10 +- .../tajo/plan/expr/RowConstantEval.java | 9 +- .../org/apache/tajo/plan/expr/SignedEval.java | 4 +- .../apache/tajo/plan/expr/SubqueryEval.java | 7 +- .../org/apache/tajo/plan/expr/UnaryEval.java | 4 +- .../tajo/plan/expr/WindowFunctionEval.java | 6 +- .../tajo/plan/serder/EvalNodeSerializer.java | 5 +- .../plan/util/EvalNodeToExprConverter.java | 2 +- .../apache/tajo/plan/util/PlannerUtil.java | 9 +- .../tajo/plan/verifier/ExprsVerifier.java | 66 +++++------ .../storage/jdbc/SQLExpressionGenerator.java | 7 +- 48 files changed, 440 insertions(+), 501 deletions(-) create mode 100644 tajo-common/src/main/java/org/apache/tajo/type/Int1.java diff --git a/CHANGES b/CHANGES index 829f9a3aa7..7ac7d832e9 100644 --- a/CHANGES +++ b/CHANGES @@ -10,6 +10,8 @@ Release 0.12.0 - unreleased IMPROVEMENT + TAJO-2127: Use Type instead of DataType for EvalNode. (hyunsik) + TAJO-2125: Detect JAVA_HOME on OSX, if JAVA_HOME is not set. (jinho) TAJO-2108: Refactor Schema to be immutable. (hyunsik) diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java index c12aa291f6..9cbdbe0391 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java @@ -19,20 +19,19 @@ package org.apache.tajo; import com.google.common.collect.Maps; -import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.util.TUtil; import java.util.Map; -import static org.apache.tajo.common.TajoDataTypes.Type; import static org.apache.tajo.common.TajoDataTypes.Type.*; +import static org.apache.tajo.type.Type.*; public class DataTypeUtil { - public static final Map> FUNCTION_ACCEPTABLE_PARAM_MAP = Maps.newHashMap(); + public static final Map> FUNCTION_ACCEPTABLE_PARAM_MAP = Maps.newHashMap(); - private static void putAcceptableType(Type given, Type define) { + private static void putAcceptableType(TajoDataTypes.Type given, TajoDataTypes.Type define) { TUtil.putToNestedMap(FUNCTION_ACCEPTABLE_PARAM_MAP, given, define, true); } @@ -75,8 +74,8 @@ private static void putAcceptableType(Type given, Type define) { putAcceptableType(INET4, INET4); } - public static boolean isUpperCastable(Type define, Type given) { - if (given == define) { + public static boolean isUpperCastable(TajoDataTypes.Type define, TajoDataTypes.Type given) { + if (given.equals(define)) { return true; } @@ -86,92 +85,94 @@ public static boolean isUpperCastable(Type define, Type given) { /** * This is verified by ExprsVerifier.checkArithmeticOperand(). */ - public static TajoDataTypes.DataType determineType(TajoDataTypes.DataType left, TajoDataTypes.DataType right) { - switch (left.getType()) { + public static org.apache.tajo.type.Type determineType(org.apache.tajo.type.Type left, + org.apache.tajo.type.Type right) { + TajoDataTypes.Type rhsBaseType = right.baseType(); + switch (left.baseType()) { case INT1: case INT2: case INT4: { - switch(right.getType()) { + switch(rhsBaseType) { case INT1: case INT2: - case INT4: return CatalogUtil.newSimpleDataType(Type.INT4); - case INT8: return CatalogUtil.newSimpleDataType(Type.INT8); - case FLOAT4: return CatalogUtil.newSimpleDataType(Type.FLOAT4); - case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8); - case DATE: return CatalogUtil.newSimpleDataType(Type.DATE); - case INTERVAL: return CatalogUtil.newSimpleDataType(Type.INTERVAL); + case INT4: return Int4; + case INT8: return Int8; + case FLOAT4: return Float4; + case FLOAT8: return Float8; + case DATE: return Date; + case INTERVAL: return Interval; } } case INT8: { - switch(right.getType()) { + switch(rhsBaseType) { case INT1: case INT2: case INT4: - case INT8: return CatalogUtil.newSimpleDataType(Type.INT8); - case FLOAT4: return CatalogUtil.newSimpleDataType(Type.FLOAT4); - case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8); - case DATE: return CatalogUtil.newSimpleDataType(Type.DATE); - case INTERVAL: return CatalogUtil.newSimpleDataType(Type.INTERVAL); + case INT8: return Int8; + case FLOAT4: return Float4; + case FLOAT8: return Float8; + case DATE: return Date; + case INTERVAL: return Interval; } } case FLOAT4: { - switch(right.getType()) { + switch(rhsBaseType) { case INT1: case INT2: - case INT4: return CatalogUtil.newSimpleDataType(Type.FLOAT4); - case INT8: return CatalogUtil.newSimpleDataType(Type.FLOAT4); - case FLOAT4: return CatalogUtil.newSimpleDataType(Type.FLOAT4); - case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8); - case INTERVAL: return CatalogUtil.newSimpleDataType(Type.INTERVAL); + case INT4: return Float4; + case INT8: return Float4; + case FLOAT4: return Float4; + case FLOAT8: return Float8; + case INTERVAL: return Interval; } } case FLOAT8: { - switch(right.getType()) { + switch(rhsBaseType) { case INT1: case INT2: case INT4: case INT8: case FLOAT4: - case FLOAT8: return CatalogUtil.newSimpleDataType(Type.FLOAT8); - case INTERVAL: return CatalogUtil.newSimpleDataType(Type.INTERVAL); + case FLOAT8: return Float8; + case INTERVAL: return Interval; } } case DATE: { - switch(right.getType()) { + switch(rhsBaseType) { case INT2: case INT4: - case INT8: return CatalogUtil.newSimpleDataType(Type.DATE); + case INT8: return Date; case INTERVAL: - case TIME: return CatalogUtil.newSimpleDataType(Type.TIMESTAMP); - case DATE: return CatalogUtil.newSimpleDataType(Type.INT4); + case TIME: return Timestamp; + case DATE: return Int4; } } case TIME: { - switch(right.getType()) { - case INTERVAL: return CatalogUtil.newSimpleDataType(Type.TIME); - case TIME: return CatalogUtil.newSimpleDataType(Type.INTERVAL); - case DATE: return CatalogUtil.newSimpleDataType(Type.INT4); + switch(rhsBaseType) { + case INTERVAL: return Time; + case TIME: return Interval; + case DATE: return Int4; } } case TIMESTAMP: { - switch (right.getType()) { - case INTERVAL: return CatalogUtil.newSimpleDataType(Type.TIMESTAMP); - case TIMESTAMP: return CatalogUtil.newSimpleDataType(Type.INTERVAL); + switch (rhsBaseType) { + case INTERVAL: return Timestamp; + case TIMESTAMP: return Interval; } } case INTERVAL: { - switch (right.getType()) { + switch (rhsBaseType) { case INTERVAL: case FLOAT4: - case FLOAT8: return CatalogUtil.newSimpleDataType(Type.INTERVAL); + case FLOAT8: return Interval; } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java index 12edaa74cb..641b674cc8 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java @@ -26,6 +26,7 @@ import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.json.GsonObject; +import org.apache.tajo.type.Type; /** * Describes a column. It is an immutable object. @@ -63,6 +64,15 @@ public Column(String name, TajoDataTypes.Type type) { this(name, CatalogUtil.newSimpleDataType(type)); } + /** + * + * @param name Column name + * @param type Type + */ + public Column(String name, Type type) { + this(name, TypeConverter.convert(type)); + } + /** * * @param name Column name diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java index f82cc03593..f4164d4434 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java @@ -20,11 +20,11 @@ import com.google.common.collect.ImmutableList; import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.exception.NotImplementedException; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.schema.Schema; -import org.apache.tajo.type.Protobuf; -import org.apache.tajo.type.Type; +import org.apache.tajo.type.*; import java.util.Collection; @@ -43,36 +43,41 @@ public static Collection convert(TypeDesc type) { public static Type convert(TajoDataTypes.Type legacyBaseType) { switch (legacyBaseType) { case BOOLEAN: - return Bool(); + return Bool; case INT1: + return Int1; case INT2: - return Int2(); + return Int2; case INT4: - return Int4(); + return Int4; case INT8: - return Int8(); + return Int8; case FLOAT4: - return Float4(); + return Float4; case FLOAT8: - return Float8(); + return Float8; case DATE: - return Date(); + return Date; case TIME: - return Time(); + return Time; case TIMESTAMP: - return Timestamp(); + return Timestamp; case INTERVAL: - return Interval(); + return Interval; + case CHAR: + return Char(1); // default len = 1 case TEXT: - return Text(); + return Text; case BLOB: - return Blob(); + return Blob; case INET4: - return Inet4(); + return Inet4; + case RECORD: + throw new TajoRuntimeException(new NotImplementedException("record projection")); case NULL_TYPE: - return Null(); + return Null; case ANY: - return Any(); + return Any; default: throw new TajoRuntimeException(new UnsupportedException(legacyBaseType.name())); } @@ -96,6 +101,21 @@ public static Type convert(TajoDataTypes.DataType legacyType) { } public static TajoDataTypes.DataType convert(Type type) { - return CatalogUtil.newSimpleDataType(type.baseType()); + switch (type.baseType()) { + case CHAR: + Char charType = (Char) type; + return CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.CHAR, charType.length()); + case VARCHAR: + Varchar varcharType = (Varchar) type; + return CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.VARCHAR, varcharType.length()); + case PROTOBUF: + Protobuf protobuf = (Protobuf) type; + return CatalogUtil.newDataType(TajoDataTypes.Type.PROTOBUF, protobuf.getMessageName()); + case NUMERIC: + Numeric numericType = (Numeric) type; + return CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.NUMERIC, numericType.precision()); + default: + return CatalogUtil.newSimpleDataType(type.baseType()); + } } } diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidValueForCastException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidValueForCastException.java index 5baf7c6af2..6af8d1a1b3 100644 --- a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidValueForCastException.java +++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidValueForCastException.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -21,19 +21,24 @@ import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.error.Errors; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState; +import org.apache.tajo.type.Type; public class InvalidValueForCastException extends TajoException { - private static final long serialVersionUID = -7689027447969916148L; + private static final long serialVersionUID = -7689027447969916148L; - public InvalidValueForCastException(ReturnState state) { - super(state); - } + public InvalidValueForCastException(ReturnState state) { + super(state); + } - public InvalidValueForCastException(TajoDataTypes.DataType src, TajoDataTypes.DataType target) { - super(Errors.ResultCode.INVALID_VALUE_FOR_CAST, src.getType().name(), target.getType().name()); - } + public InvalidValueForCastException(TajoDataTypes.DataType src, TajoDataTypes.DataType target) { + super(Errors.ResultCode.INVALID_VALUE_FOR_CAST, src.getType().name(), target.getType().name()); + } public InvalidValueForCastException(TajoDataTypes.Type src, TajoDataTypes.Type target) { super(Errors.ResultCode.INVALID_VALUE_FOR_CAST, src.name(), target.name()); } + + public InvalidValueForCastException(Type src, Type target) { + super(Errors.ResultCode.INVALID_VALUE_FOR_CAST, src.baseType().name(), target.baseType().name()); + } } diff --git a/tajo-common/src/main/java/org/apache/tajo/type/Int1.java b/tajo-common/src/main/java/org/apache/tajo/type/Int1.java new file mode 100644 index 0000000000..9288328d36 --- /dev/null +++ b/tajo-common/src/main/java/org/apache/tajo/type/Int1.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.type; + +import org.apache.tajo.common.TajoDataTypes; + +public class Int1 extends Type { + @Override + public TajoDataTypes.Type baseType() { + return TajoDataTypes.Type.INT1; + } +} diff --git a/tajo-common/src/main/java/org/apache/tajo/type/Type.java b/tajo-common/src/main/java/org/apache/tajo/type/Type.java index e1a7180a33..fdea07e0ed 100644 --- a/tajo-common/src/main/java/org/apache/tajo/type/Type.java +++ b/tajo-common/src/main/java/org/apache/tajo/type/Type.java @@ -25,6 +25,24 @@ public abstract class Type { + // No paramter types + public static final Any Any = new Any(); + public static final Null Null = new Null(); + public static final Bool Bool = new Bool(); + public static final Int1 Int1 = new Int1(); + public static final Int2 Int2 = new Int2(); + public static final Int4 Int4 = new Int4(); + public static final Int8 Int8 = new Int8(); + public static final Float4 Float4 = new Float4(); + public static final Float8 Float8 = new Float8(); + public static final Date Date = new Date(); + public static final Time Time = new Time(); + public static final Timestamp Timestamp = new Timestamp(); + public static final Interval Interval = new Interval(); + public static final Text Text = new Text(); + public static final Blob Blob = new Blob(); + public static final Inet4 Inet4 = new Inet4(); + public abstract TajoDataTypes.Type baseType(); public boolean hasParam() { @@ -42,10 +60,7 @@ public int hashCode() { @Override public boolean equals(Object t) { - if (t instanceof Type) { - return ((Type)t).baseType() == baseType(); - } - return false; + return t instanceof Type && ((Type)t).baseType() == baseType(); } @Override @@ -57,33 +72,7 @@ public boolean isStruct() { return this.baseType() == TajoDataTypes.Type.RECORD; } - public static Any Any() { - return Any.INSTANCE; - } - - public static Bool Bool() { - return new Bool(); - } - - public static Int2 Int2() { - return new Int2(); - } - - public static Int4 Int4() { - return new Int4(); - } - - public static Int8 Int8() { - return new Int8(); - } - - public static Float4 Float4() { - return new Float4(); - } - - public static Float8 Float8() { - return new Float8(); - } + public boolean isNull() { return this.baseType() == TajoDataTypes.Type.NULL_TYPE; } public static int DEFAULT_SCALE = 0; diff --git a/tajo-common/src/test/java/org/apache/tajo/schema/TestSchema.java b/tajo-common/src/test/java/org/apache/tajo/schema/TestSchema.java index 03cc223359..a843bd95d0 100644 --- a/tajo-common/src/test/java/org/apache/tajo/schema/TestSchema.java +++ b/tajo-common/src/test/java/org/apache/tajo/schema/TestSchema.java @@ -32,8 +32,8 @@ public class TestSchema { @Test public final void testSchema1() { - NamedType struct1 = Struct($("f12"), Field($("f1"), Int8()), Field($("f2"), Int4())); - NamedType struct2 = Struct($("f34"), Field($("f3"), Int8()), Field($("f4"), Int4())); + NamedType struct1 = Struct($("f12"), Field($("f1"), Int8), Field($("f2"), Int4)); + NamedType struct2 = Struct($("f34"), Field($("f3"), Int8), Field($("f4"), Int4)); Schema schema = Schema(struct1, struct2); assertEquals(schema.toString(), "f12 record (f1 int8,f2 int4),f34 record (f3 int8,f4 int4)"); @@ -41,9 +41,9 @@ public final void testSchema1() { @Test public final void testSchema2() { - NamedType f1 = Field($("x"), Array(Int8())); - NamedType f2 = Field($("y"), Int8()); - NamedType f3 = Struct($("z"), Field($("z-1"), Time()), Field($("z-2"), Array(Int8()))); + NamedType f1 = Field($("x"), Array(Int8)); + NamedType f2 = Field($("y"), Int8); + NamedType f3 = Struct($("z"), Field($("z-1"), Time()), Field($("z-2"), Array(Int8))); Schema schema = Schema(f1, f2, f3); assertEquals(schema.toString(), "x array,y int8,z record (z-1 time,z-2 array)"); @@ -51,9 +51,9 @@ public final void testSchema2() { @Test public final void testSchemaWithIdentifiers() { - NamedType f1 = Field($("x", "y"), Array(Int8())); - NamedType f2 = Field($(_("y"), _("B", true)), Int8()); - NamedType f3 = Struct($("z"), Field($("z-1"), Time()), Field($("z-2"), Array(Int8()))); + NamedType f1 = Field($("x", "y"), Array(Int8)); + NamedType f2 = Field($(_("y"), _("B", true)), Int8); + NamedType f3 = Struct($("z"), Field($("z-1"), Time), Field($("z-2"), Array(Int8))); Schema schema = Schema(f1, f2, f3); assertEquals(schema.toString(), "x.y array,y.'B' int8,z record (z-1 time,z-2 array)"); diff --git a/tajo-common/src/test/java/org/apache/tajo/type/TestType.java b/tajo-common/src/test/java/org/apache/tajo/type/TestType.java index 9023af0af2..910ba0db22 100644 --- a/tajo-common/src/test/java/org/apache/tajo/type/TestType.java +++ b/tajo-common/src/test/java/org/apache/tajo/type/TestType.java @@ -27,15 +27,16 @@ public class TestType { @Test public final void testPrimitiveTypes() { - assertEquals(Bool().baseType(), BOOLEAN); - assertEquals(Int2().baseType(), INT2); - assertEquals(Int4().baseType(), INT4); - assertEquals(Int8().baseType(), INT8); - assertEquals(Float4().baseType(), FLOAT4); - assertEquals(Float8().baseType(), FLOAT8); - assertEquals(Date().baseType(), DATE); - assertEquals(Time().baseType(), TIME); - assertEquals(Timestamp().baseType(), TIMESTAMP); + assertEquals(Bool.baseType(), BOOLEAN); + assertEquals(Int1.baseType(), INT1); + assertEquals(Int2.baseType(), INT2); + assertEquals(Int4.baseType(), INT4); + assertEquals(Int8.baseType(), INT8); + assertEquals(Float4.baseType(), FLOAT4); + assertEquals(Float8.baseType(), FLOAT8); + assertEquals(Date.baseType(), DATE); + assertEquals(Time.baseType(), TIME); + assertEquals(Timestamp.baseType(), TIMESTAMP); Numeric n = Numeric(4, 2); assertEquals(n.baseType(), NUMERIC); @@ -52,37 +53,38 @@ public final void testPrimitiveTypes() { assertEquals(varchar.baseType(), VARCHAR); assertEquals(varchar.length(), 2); - Struct struct = Struct(Int8(), Array(Float8())); + Struct struct = Struct(Int8, Array(Float8)); assertEquals(struct.baseType(), RECORD); assertEquals(struct.memberType(0).baseType(), INT8); assertEquals(struct.memberType(1).baseType(), ARRAY); - Map map = Map(Int8(), Array(Timestamp())); + Map map = Map(Int8, Array(Timestamp())); assertEquals(map.baseType(), MAP); assertEquals(map.keyType().baseType(), INT8); assertEquals(map.valueType().baseType(), ARRAY); - Array array = Array(Int8()); + Array array = Array(Int8); assertEquals(array.baseType(), ARRAY); assertEquals(array.elementType().baseType(), INT8); } @Test public final void testToString() { - assertEquals("boolean", Bool().toString()); - assertEquals("int2", Int2().toString()); - assertEquals("int4", Int4().toString()); - assertEquals("int8", Int8().toString()); - assertEquals("float4", Float4().toString()); - assertEquals("float8", Float8().toString()); - assertEquals("date", Date().toString()); - assertEquals("time", Time().toString()); - assertEquals("timestamp", Timestamp().toString()); + assertEquals("boolean", Bool.toString()); + assertEquals("int1", Int1.toString()); + assertEquals("int2", Int2.toString()); + assertEquals("int4", Int4.toString()); + assertEquals("int8", Int8.toString()); + assertEquals("float4", Float4.toString()); + assertEquals("float8", Float8.toString()); + assertEquals("date", Date.toString()); + assertEquals("time", Time.toString()); + assertEquals("timestamp", Timestamp.toString()); Numeric n = Numeric(4, 2); assertEquals("numeric(4,2)", n.toString()); - assertEquals("blob", Blob().toString()); + assertEquals("blob", Blob.toString()); Char c = Char(2); assertEquals("char(2)", c.toString()); @@ -90,13 +92,13 @@ public final void testToString() { Varchar varchar = Varchar(2); assertEquals("varchar(2)", varchar.toString()); - Struct struct = Struct(Int8(), Array(Float8())); + Struct struct = Struct(Int8, Array(Float8)); assertEquals("struct(int8,array)", struct.toString()); - Map map = Map(Int8(), Array(Timestamp())); + Map map = Map(Int8, Array(Timestamp)); assertEquals("map>", map.toString()); - Array array = Array(Int8()); + Array array = Array(Int8); assertEquals("array", array.toString()); } } \ No newline at end of file diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 61faee619e..00fa590055 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -33,7 +33,6 @@ import org.apache.tajo.engine.codegen.EvalCodeGenerator; import org.apache.tajo.engine.codegen.TajoClassLoader; import org.apache.tajo.engine.function.FunctionLoader; -import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.exception.TajoException; import org.apache.tajo.exception.TajoInternalError; @@ -119,12 +118,6 @@ public static void tearDown() throws Exception { cluster.shutdownCatalogCluster(); } - private static void assertJsonSerDer(EvalNode expr) { - String json = CoreGsonHelper.toJson(expr, EvalNode.class); - EvalNode fromJson = CoreGsonHelper.fromJson(json, EvalNode.class); - assertEquals(expr, fromJson); - } - public TajoConf getConf() { return new TajoConf(conf); } @@ -174,9 +167,6 @@ private static List getRawTargets(QueryContext context, String query, bo fail(e.getMessage()); } } - for (Target t : targets) { - assertJsonSerDer(t.getEvalTree()); - } for (Target t : targets) { assertEvalTreeProtoSerDer(context, t.getEvalTree()); } diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java index e74b108f29..26e0615e52 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java @@ -23,16 +23,15 @@ import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.SchemaBuilder; import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.plan.expr.*; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; +import org.apache.tajo.type.Type; import org.junit.Test; -import static org.apache.tajo.common.TajoDataTypes.Type.*; +import static org.apache.tajo.common.TajoDataTypes.Type.INT4; import static org.junit.Assert.*; public class TestEvalTree extends ExprTestBase { @@ -93,8 +92,8 @@ public boolean equals(Object obj) { } @Override - public DataType getValueType() { - return CatalogUtil.newSimpleDataType(BOOLEAN); + public Type getValueType() { + return Type.Bool; } @Override @@ -141,8 +140,8 @@ public void postOrder(EvalNodeVisitor visitor) { } @Override - public DataType getValueType() { - return CatalogUtil.newSimpleDataType(BOOLEAN); + public Type getValueType() { + return Type.Bool; } @Override @@ -291,16 +290,16 @@ public final void testGetReturnType() { e1 = new ConstEval(DatumFactory.createInt4(9)); e2 = new ConstEval(DatumFactory.createInt4(34)); BinaryEval expr = new BinaryEval(EvalType.PLUS, e1, e2); - assertEquals(CatalogUtil.newSimpleDataType(INT4), expr.getValueType()); + assertEquals(Type.Int4, expr.getValueType()); expr = new BinaryEval(EvalType.LTH, e1, e2); assertTrue(expr.bind(null, null).eval(null).asBool()); - assertEquals(CatalogUtil.newSimpleDataType(BOOLEAN), expr.getValueType()); + assertEquals(Type.Bool, expr.getValueType()); e1 = new ConstEval(DatumFactory.createFloat8(9.3)); e2 = new ConstEval(DatumFactory.createFloat8(34.2)); expr = new BinaryEval(EvalType.PLUS, e1, e2); - assertEquals(CatalogUtil.newSimpleDataType(FLOAT8), expr.getValueType()); + assertEquals(Type.Float8, expr.getValueType()); } @Test @@ -334,42 +333,6 @@ public final void testEquals() throws CloneNotSupportedException { assertTrue(compExpr1.equals(compExpr2)); } - - @Test - public final void testJson() throws CloneNotSupportedException { - ConstEval e1; - ConstEval e2; - - // 29 > (34 + 5) + (5 + 34) - e1 = new ConstEval(DatumFactory.createInt4(34)); - e2 = new ConstEval(DatumFactory.createInt4(5)); - assertCloneEqual(e1); - - BinaryEval plus1 = new BinaryEval(EvalType.PLUS, e1, e2); - assertCloneEqual(plus1); - BinaryEval plus2 = new BinaryEval(EvalType.PLUS, e2, e1); - assertCloneEqual(plus2); - BinaryEval plus3 = new BinaryEval(EvalType.PLUS, plus2, plus1); - assertCloneEqual(plus3); - - ConstEval e3 = new ConstEval(DatumFactory.createInt4(29)); - BinaryEval gth = new BinaryEval(EvalType.GTH, e3, plus3); - assertCloneEqual(gth); - - String json = gth.toJson(); - BinaryEval eval = (BinaryEval) CoreGsonHelper.fromJson(json, EvalNode.class); - assertCloneEqual(eval); - - assertEquals(gth.getType(), eval.getType()); - assertEquals(e3.getType(), eval.getLeftExpr().getType()); - assertEquals(plus3.getType(), eval.getRightExpr().getType()); - assertEquals(plus3.getLeftExpr(), ((BinaryEval)eval.getRightExpr()).getLeftExpr()); - assertEquals(plus3.getRightExpr(), ((BinaryEval) eval.getRightExpr()).getRightExpr()); - assertEquals(plus2.getLeftExpr(), ((BinaryEval)((BinaryEval)eval.getRightExpr()).getLeftExpr()).getLeftExpr()); - assertEquals(plus2.getRightExpr(), ((BinaryEval)((BinaryEval)eval.getRightExpr()).getLeftExpr()).getRightExpr()); - assertEquals(plus1.getLeftExpr(), ((BinaryEval) ((BinaryEval) eval.getRightExpr()).getRightExpr()).getLeftExpr()); - assertEquals(plus1.getRightExpr(), ((BinaryEval) ((BinaryEval) eval.getRightExpr()).getRightExpr()).getRightExpr()); - } @Test public final void testBindCheck() { diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java index d1c83844dd..d01b0d53b5 100644 --- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java +++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java @@ -187,7 +187,6 @@ public final void testSingleRelation() throws CloneNotSupportedException, TajoEx assertEquals(NodeType.ROOT, plan.getType()); testCloneLogicalNode(plan); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); assertEquals(NodeType.PROJECTION, root.getChild().getType()); ProjectionNode projNode = root.getChild(); @@ -222,7 +221,6 @@ public final void testImplicityJoinPlan() throws CloneNotSupportedException, Taj assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); testCloneLogicalNode(root); Schema expectedSchema = SchemaBuilder.builder() @@ -252,7 +250,6 @@ public final void testImplicityJoinPlan() throws CloneNotSupportedException, Taj // three relations expr = sqlAnalyzer.parse(QUERIES[2]); plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); testCloneLogicalNode(plan); expectedSchema = SchemaBuilder.builder().addAll(expectedSchema.getRootColumns()) @@ -310,7 +307,6 @@ public final void testNaturalJoinPlan() throws TajoException { // two relations Expr context = sqlAnalyzer.parse(JOINS[0]); LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertSchema(expectedJoinSchema, plan.getOutSchema()); assertEquals(NodeType.ROOT, plan.getType()); @@ -343,7 +339,6 @@ public final void testInnerJoinPlan() throws TajoException { Expr expr = sqlAnalyzer.parse(JOINS[1]); LogicalPlan plan = planner.createPlan(qc, expr); LogicalNode root = plan.getRootBlock().getRoot(); - testJsonSerDerObject(root); assertSchema(expectedJoinSchema, root.getOutSchema()); assertEquals(NodeType.ROOT, root.getType()); @@ -376,7 +371,6 @@ public final void testOuterJoinPlan() throws TajoException { // two relations Expr expr = sqlAnalyzer.parse(JOINS[2]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertSchema(expectedJoinSchema, plan.getOutSchema()); assertEquals(NodeType.ROOT, plan.getType()); @@ -414,7 +408,6 @@ public final void testGroupby() throws CloneNotSupportedException, TajoException assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); testQuery7(root.getChild()); // with having clause @@ -452,7 +445,6 @@ public final void testMultipleJoin() throws IOException, TajoException { FileUtil.readTextFile(new File("src/test/resources/queries/TestJoinQuery/testTPCHQ2Join.sql"))); QueryContext qc = createQueryContext(); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); Schema expected = tpch.getOutSchema("q2"); assertSchema(expected, plan.getOutSchema()); } @@ -513,7 +505,6 @@ public final void testJoinWithMultipleJoinQual1() throws IOException, TajoExcept LogicalPlan plan = planner.createPlan(qc, expr); LogicalNode node = plan.getRootBlock().getRoot(); - testJsonSerDerObject(node); Schema expected = tpch.getOutSchema("q2"); assertSchema(expected, node.getOutSchema()); @@ -555,7 +546,6 @@ public final void testJoinWithMultipleJoinQual2() throws IOException, TajoExcept LogicalPlan plan = planner.createPlan(qc,expr); LogicalNode node = plan.getRootBlock().getRoot(); - testJsonSerDerObject(node); LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration(), catalog, TablespaceManager.getInstance()); optimizer.optimize(plan); @@ -596,7 +586,6 @@ public final void testJoinWithMultipleJoinQual3() throws IOException, TajoExcept LogicalPlan plan = planner.createPlan(qc, expr); LogicalNode node = plan.getRootBlock().getRoot(); - testJsonSerDerObject(node); LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration(), catalog, TablespaceManager.getInstance()); optimizer.optimize(plan); @@ -643,7 +632,6 @@ public final void testJoinWithMultipleJoinQual4() throws IOException, TajoExcept LogicalPlan plan = planner.createPlan(qc, expr); LogicalNode node = plan.getRootBlock().getRoot(); - testJsonSerDerObject(node); LogicalOptimizer optimizer = new LogicalOptimizer(util.getConfiguration(), catalog, TablespaceManager.getInstance()); optimizer.optimize(plan); @@ -736,7 +724,6 @@ public final void testStoreTable() throws CloneNotSupportedException, TajoExcept LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot(); testCloneLogicalNode(plan); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; @@ -753,7 +740,6 @@ public final void testOrderBy() throws CloneNotSupportedException, TajoException Expr expr = sqlAnalyzer.parse(QUERIES[4]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); testCloneLogicalNode(plan); assertEquals(NodeType.ROOT, plan.getType()); @@ -783,7 +769,6 @@ public final void testLimit() throws CloneNotSupportedException, TajoException { Expr expr = sqlAnalyzer.parse(QUERIES[12]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); testCloneLogicalNode(plan); assertEquals(NodeType.ROOT, plan.getType()); @@ -804,7 +789,6 @@ public final void testSPJPush() throws CloneNotSupportedException, TajoException Expr expr = sqlAnalyzer.parse(QUERIES[5]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); testCloneLogicalNode(plan); assertEquals(NodeType.ROOT, plan.getType()); @@ -826,32 +810,9 @@ public final void testSPJ() throws CloneNotSupportedException, TajoException { Expr expr = sqlAnalyzer.parse(QUERIES[6]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); testCloneLogicalNode(plan); } - @Test - public final void testJson() throws TajoException { - QueryContext qc = createQueryContext(); - - Expr expr = sqlAnalyzer.parse(QUERIES[9]); - LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); - - String json = plan.toJson(); - LogicalNode fromJson = CoreGsonHelper.fromJson(json, LogicalNode.class); - assertEquals(NodeType.ROOT, fromJson.getType()); - LogicalNode project = ((LogicalRootNode)fromJson).getChild(); - assertEquals(NodeType.PROJECTION, project.getType()); - assertEquals(NodeType.HAVING, ((ProjectionNode) project).getChild().getType()); - HavingNode havingNode = ((ProjectionNode) project).getChild(); - assertEquals(NodeType.GROUP_BY, havingNode.getChild().getType()); - GroupbyNode groupbyNode = havingNode.getChild(); - assertEquals(NodeType.SCAN, groupbyNode.getChild().getType()); - LogicalNode scan = groupbyNode.getChild(); - assertEquals(NodeType.SCAN, scan.getType()); - } - @Test public final void testVisitor() throws TajoException { QueryContext qc = createQueryContext(); @@ -886,7 +847,6 @@ public final void testExprNode() throws TajoException { Expr expr = sqlAnalyzer.parse(QUERIES[10]); LogicalPlan rootNode = planner.createPlan(qc, expr); LogicalNode plan = rootNode.getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; assertEquals(NodeType.EXPRS, root.getChild().getType()); @@ -907,7 +867,6 @@ public final void testCreateIndexNode() throws TajoException { Expr expr = sqlAnalyzer.parse(QUERIES[11]); LogicalPlan rootNode = planner.createPlan(qc, expr); LogicalNode plan = rootNode.getRootBlock().getRoot(); - testJsonSerDerObject(plan); LogicalRootNode root = (LogicalRootNode) plan; assertEquals(NodeType.CREATE_INDEX, root.getChild().getType()); @@ -934,7 +893,6 @@ public final void testAsterisk() throws CloneNotSupportedException, TajoExceptio assertEquals(NodeType.ROOT, plan.getType()); testCloneLogicalNode(plan); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); assertEquals(NodeType.PROJECTION, root.getChild().getType()); ProjectionNode projNode = root.getChild(); @@ -961,7 +919,6 @@ public final void testAlias1() throws TajoException { Expr expr = sqlAnalyzer.parse(ALIAS[0]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); Schema finalSchema = root.getOutSchema(); Iterator it = finalSchema.getRootColumns().iterator(); @@ -989,7 +946,6 @@ public final void testAlias2() throws TajoException { Expr expr = sqlAnalyzer.parse(ALIAS[1]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); Schema finalSchema = root.getOutSchema(); Iterator it = finalSchema.getRootColumns().iterator(); @@ -1010,7 +966,6 @@ public final void testCreateTableDef() throws TajoException { Expr expr = sqlAnalyzer.parse(CREATE_TABLE[0]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); LogicalRootNode root = (LogicalRootNode) plan; - testJsonSerDerObject(root); assertEquals(NodeType.CREATE_TABLE, root.getChild().getType()); CreateTableNode createTable = root.getChild(); @@ -1095,7 +1050,6 @@ public final void testSetPlan() throws TajoException { Expr expr = sqlAnalyzer.parse(setStatements[0]); LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; assertEquals(NodeType.UNION, root.getChild().getType()); @@ -1116,7 +1070,6 @@ public void testSetQualifier() throws TajoException { Expr context = sqlAnalyzer.parse(setQualifiers[0]); LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; assertEquals(NodeType.PROJECTION, root.getChild().getType()); @@ -1125,7 +1078,6 @@ public void testSetQualifier() throws TajoException { context = sqlAnalyzer.parse(setQualifiers[1]); plan = planner.createPlan(qc, context).getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); root = (LogicalRootNode) plan; assertEquals(NodeType.PROJECTION, root.getChild().getType()); @@ -1134,19 +1086,12 @@ public void testSetQualifier() throws TajoException { context = sqlAnalyzer.parse(setQualifiers[2]); plan = planner.createPlan(qc, context).getRootBlock().getRoot(); - testJsonSerDerObject(plan); root = (LogicalRootNode) plan; assertEquals(NodeType.PROJECTION, root.getChild().getType()); projectionNode = root.getChild(); assertEquals(NodeType.SCAN, projectionNode.getChild().getType()); } - public void testJsonSerDerObject(LogicalNode rootNode) { - String json = rootNode.toJson(); - LogicalNode fromJson = CoreGsonHelper.fromJson(json, LogicalNode.class); - assertTrue("JSON (de) serialization equivalence check", rootNode.deepEquals(fromJson)); - } - // Table descriptions // // employee (name text, empid int4, deptname text) @@ -1275,7 +1220,6 @@ public final void testAlterTableRepairPartiton() throws TajoException { Expr expr = sqlAnalyzer.parse(sql); LogicalPlan rootNode = planner.createPlan(qc, expr); LogicalNode plan = rootNode.getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; assertEquals(NodeType.ALTER_TABLE, root.getChild().getType()); @@ -1341,7 +1285,6 @@ public final void testAddPartitionAndDropPartition() throws TajoException { Expr expr = sqlAnalyzer.parse(ALTER_PARTITIONS[0]); LogicalPlan rootNode = planner.createPlan(qc, expr); LogicalNode plan = rootNode.getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); LogicalRootNode root = (LogicalRootNode) plan; assertEquals(NodeType.ALTER_TABLE, root.getChild().getType()); @@ -1365,7 +1308,6 @@ public final void testAddPartitionAndDropPartition() throws TajoException { expr = sqlAnalyzer.parse(ALTER_PARTITIONS[1]); rootNode = planner.createPlan(qc, expr); plan = rootNode.getRootBlock().getRoot(); - testJsonSerDerObject(plan); assertEquals(NodeType.ROOT, plan.getType()); root = (LogicalRootNode) plan; assertEquals(NodeType.ALTER_TABLE, root.getChild().getType()); @@ -1407,12 +1349,10 @@ public void testSelectFromSelfDescTable() throws Exception { LogicalNode node = logicalPlan.getRootNode(); assertEquals(NodeType.ROOT, node.getType()); LogicalRootNode root = (LogicalRootNode) node; - testJsonSerDerObject(root); testCloneLogicalNode(root); assertEquals(NodeType.PROJECTION, root.getChild().getType()); ProjectionNode projectionNode = root.getChild(); - testJsonSerDerObject(projectionNode); testCloneLogicalNode(projectionNode); // projection column test @@ -1462,12 +1402,10 @@ public void testSelectWhereFromSelfDescTable() throws Exception { LogicalNode node = logicalPlan.getRootNode(); assertEquals(NodeType.ROOT, node.getType()); LogicalRootNode root = (LogicalRootNode) node; - testJsonSerDerObject(root); testCloneLogicalNode(root); assertEquals(NodeType.PROJECTION, root.getChild().getType()); ProjectionNode projectionNode = root.getChild(); - testJsonSerDerObject(projectionNode); testCloneLogicalNode(projectionNode); // projection column test diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java index ca3466bd2d..0dea1044cc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java @@ -116,7 +116,7 @@ public void emitConstructor() { if (entry.getKey().getType() == EvalType.CONST) { ConstEval constEval = (ConstEval) entry.getKey(); - if (constEval.getValueType().getType() == TajoDataTypes.Type.INTERVAL) { + if (constEval.getValueType().baseType() == TajoDataTypes.Type.INTERVAL) { IntervalDatum datum = (IntervalDatum) constEval.getValue(); final String internalName = TajoGeneratorAdapter.getInternalName(IntervalDatum.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java index 1bf0d573d4..5ee6d82efa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java @@ -20,7 +20,6 @@ import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.IntervalDatum; import org.apache.tajo.datum.ProtobufDatum; @@ -36,7 +35,6 @@ import java.lang.reflect.Constructor; import java.util.Stack; -import static org.apache.tajo.common.TajoDataTypes.DataType; import static org.apache.tajo.engine.codegen.TajoGeneratorAdapter.getDescription; import static org.apache.tajo.plan.expr.FunctionEval.ParamType; @@ -172,7 +170,7 @@ public EvalNode visitUnaryEval(EvalCodeGenContext context, UnaryEval unary, Stac context.emitNullityCheck(ifNull); SignedEval signed = (SignedEval) unary; - switch (signed.getValueType().getType()) { + switch (signed.getValueType().baseType()) { case BOOLEAN: case CHAR: case INT1: @@ -322,8 +320,8 @@ void emitLabel(EvalCodeGenContext context, Label label) { } public EvalNode visitCast(EvalCodeGenContext context, Stack stack, CastEval cast) { - DataType srcType = cast.getOperand().getValueType(); - DataType targetType = cast.getValueType(); + org.apache.tajo.type.Type srcType = cast.getOperand().getValueType(); + org.apache.tajo.type.Type targetType = cast.getValueType(); if (srcType.equals(targetType)) { visit(context, cast.getChild(), stack); @@ -351,7 +349,7 @@ public EvalNode visitCast(EvalCodeGenContext context, Stack stack, Cas public EvalNode visitField(EvalCodeGenContext context, FieldEval field, Stack stack) { - if (field.getValueType().getType() == TajoDataTypes.Type.NULL_TYPE) { + if (field.getValueType().isNull()) { context.pushNullOfThreeValuedLogic(); context.pushNullFlag(false); } else { @@ -377,7 +375,7 @@ public EvalNode visitField(EvalCodeGenContext context, FieldEval field, Stack stack) throws CompilationError { - DataType lhsType = evalNode.getLeftExpr().getValueType(); - DataType rhsType = evalNode.getRightExpr().getValueType(); + org.apache.tajo.type.Type lhsType = evalNode.getLeftExpr().getValueType(); + org.apache.tajo.type.Type rhsType = evalNode.getRightExpr().getValueType(); - if (lhsType.getType() == TajoDataTypes.Type.NULL_TYPE || rhsType.getType() == TajoDataTypes.Type.NULL_TYPE) { + if (lhsType.isNull() || rhsType.isNull()) { context.pushNullOfThreeValuedLogic(); context.pushNullFlag(false); } else { @@ -644,7 +642,7 @@ public EvalNode visitIsNull(EvalCodeGenContext context, IsNullEval isNullEval, S @Override public EvalNode visitConst(EvalCodeGenContext context, ConstEval constEval, Stack stack) { - switch (constEval.getValueType().getType()) { + switch (constEval.getValueType().baseType()) { case NULL_TYPE: if (stack.isEmpty()) { @@ -697,11 +695,10 @@ public EvalNode visitConst(EvalCodeGenContext context, ConstEval constEval, Stac emitGetField(context, context.owner, context.symbols.get(constEval), IntervalDatum.class); break; default: - throw new UnsupportedOperationException(constEval.getValueType().getType().name() + - " const type is not supported"); + throw new UnsupportedOperationException(constEval.getValueType() + " const type is not supported"); } - context.pushNullFlag(constEval.getValueType().getType() != TajoDataTypes.Type.NULL_TYPE); + context.pushNullFlag(!constEval.getValueType().isNull()); return constEval; } @@ -709,7 +706,7 @@ public static ParamType[] getParamTypes(EvalNode [] arguments) { ParamType[] paramTypes = new ParamType[arguments.length]; for (int i = 0; i < arguments.length; i++) { if (arguments[i].getType() == EvalType.CONST) { - if (arguments[i].getValueType().getType() == TajoDataTypes.Type.NULL_TYPE) { + if (arguments[i].getValueType().isNull()) { paramTypes[i] = ParamType.NULL; } else { paramTypes[i] = ParamType.CONSTANT; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/ScalarFunctionBindingEmitter.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/ScalarFunctionBindingEmitter.java index b33a7c4700..9f022ce58d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/ScalarFunctionBindingEmitter.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/ScalarFunctionBindingEmitter.java @@ -26,6 +26,7 @@ import org.apache.tajo.org.objectweb.asm.Opcodes; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.expr.FunctionEval; +import org.apache.tajo.type.Type; import java.util.Stack; @@ -114,7 +115,7 @@ public static void emit(EvalCodeGenerator generator, EvalCodeGenContext context, } } - private static void emitFunctionReturnValue(EvalCodeGenContext context, TajoDataTypes.DataType returnType, + private static void emitFunctionReturnValue(EvalCodeGenContext context, Type returnType, StaticMethodInvocationDesc method) { if (FunctionUtil.isNullableParam(method.getReturnClass())) { Label ifNull = new Label(); @@ -145,7 +146,7 @@ private static void updateNullFlag(EvalCodeGenContext context, Class clazz, int context.istore(nullFlagId); } - private static void emitBoxedParameter(EvalCodeGenContext context, TajoDataTypes.DataType dataType) { + private static void emitBoxedParameter(EvalCodeGenContext context, Type dataType) { Label ifNull = new Label(); Label afterAll = new Label(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/TajoGeneratorAdapter.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/TajoGeneratorAdapter.java index b1a15ae9ee..8a958bbed7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/TajoGeneratorAdapter.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/TajoGeneratorAdapter.java @@ -33,6 +33,7 @@ import org.apache.tajo.org.objectweb.asm.commons.TableSwitchGenerator; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.expr.EvalType; +import org.apache.tajo.type.Char; import org.apache.tajo.util.TUtil; import org.apache.tajo.util.datetime.DateTimeUtil; @@ -160,13 +161,14 @@ public TajoGeneratorAdapter(int access, MethodVisitor methodVisitor, String name generatorAdapter = new GeneratorAdapter(methodVisitor, access, name, desc); } - public static boolean isJVMInternalInt(TajoDataTypes.DataType dataType) { - TajoDataTypes.Type type = dataType.getType(); - return type == BOOLEAN || type == INT1 || type == INT2 || type == INT4 || type== INET4; + public static boolean isJVMInternalInt(org.apache.tajo.type.Type type) { + final TajoDataTypes.Type baseType = type.baseType(); + return baseType == BOOLEAN || baseType == INT1 || baseType == INT2 || baseType == INT4 || baseType== INET4; } - public static int getWordSize(TajoDataTypes.DataType type) { - if (type.getType() == INT8 || type.getType() == FLOAT8 || type.getType() == TIMESTAMP || type.getType() == TIME) { + public static int getWordSize(org.apache.tajo.type.Type type) { + final TajoDataTypes.Type baseType = type.baseType(); + if (baseType == INT8 || baseType == FLOAT8 || baseType == TIMESTAMP || baseType == TIME) { return 2; } else { return 1; @@ -220,9 +222,9 @@ public void push(final String value) { methodvisitor.visitLdcInsn(value); } - public void ifCmp(TajoDataTypes.DataType dataType, EvalType evalType, Label elseLabel) { + public void ifCmp(org.apache.tajo.type.Type type, EvalType evalType, Label elseLabel) { - if (isJVMInternalInt(dataType)) { + if (isJVMInternalInt(type)) { switch (evalType) { case EQUAL: methodvisitor.visitJumpInsn(Opcodes.IF_ICMPNE, elseLabel); @@ -247,10 +249,10 @@ public void ifCmp(TajoDataTypes.DataType dataType, EvalType evalType, Label else } } else { - if (dataType.getType() == TEXT) { + if (type.baseType() == TEXT) { invokeVirtual(String.class, "compareTo", int.class, new Class[]{String.class}); } else { - int opCode = TajoGeneratorAdapter.getOpCode(evalType, dataType); + int opCode = TajoGeneratorAdapter.getOpCode(evalType, type); methodvisitor.visitInsn(opCode); } @@ -279,8 +281,8 @@ public void ifCmp(TajoDataTypes.DataType dataType, EvalType evalType, Label else } } - public void load(TajoDataTypes.DataType dataType, int idx) { - switch (dataType.getType()) { + public void load(org.apache.tajo.type.Type type, int idx) { + switch (type.baseType()) { case NULL_TYPE: case BOOLEAN: case CHAR: @@ -305,7 +307,7 @@ public void load(TajoDataTypes.DataType dataType, int idx) { methodvisitor.visitVarInsn(Opcodes.ALOAD, idx); break; default: - throw new CompilationError("Unknown data type: " + dataType.getType().name()); + throw new CompilationError("Unknown data type: " + type); } } @@ -398,20 +400,22 @@ public void emitNullityCheck(Label ifNull, int ... varIds) { emitNullityCheck(ifNull); } - public void pushDummyValue(TajoDataTypes.DataType type) { - if (type.getType() == NULL_TYPE) { + public void pushDummyValue(org.apache.tajo.type.Type type) { + TajoDataTypes.Type baseType = type.baseType(); + + if (type.isNull()) { pushNullOfThreeValuedLogic(); - } else if (isJVMInternalInt(type) || type.getType() == DATE) { + } else if (isJVMInternalInt(type) || baseType == DATE) { push(0); - } else if (type.getType() == TajoDataTypes.Type.INT8 || type.getType() == TIMESTAMP || type.getType() == TIME) { + } else if (baseType == INT8 || baseType == TIMESTAMP || baseType == TIME) { push(0L); - } else if (type.getType() == TajoDataTypes.Type.FLOAT8) { + } else if (baseType == FLOAT8) { push(0.0d); - } else if (type.getType() == TajoDataTypes.Type.FLOAT4) { + } else if (baseType == FLOAT4) { push(0.0f); - } else if (type.getType() == TajoDataTypes.Type.CHAR || type.getType() == TajoDataTypes.Type.TEXT) { + } else if (baseType == CHAR || baseType == TEXT) { push(""); - } else if (type.getType() == INTERVAL || type.getType() == PROTOBUF) { + } else if (baseType == INTERVAL || baseType == PROTOBUF) { invokeStatic(NullDatum.class, "get", NullDatum.class, new Class[]{}); } else { assert false; @@ -443,25 +447,26 @@ public void invokeInterface(Class owner, String methodName, Class returnType, Cl getMethodDescription(returnType, paramTypes)); } - public static boolean isPrimitiveOpCode(EvalType evalType, TajoDataTypes.DataType returnType) { - return TUtil.containsInNestedMap(OpCodesMap, evalType, returnType.getType()); + public static boolean isPrimitiveOpCode(EvalType evalType, org.apache.tajo.type.Type returnType) { + return TUtil.containsInNestedMap(OpCodesMap, evalType, returnType.baseType()); } - public static int getOpCode(EvalType evalType, TajoDataTypes.DataType returnType) { + public static int getOpCode(EvalType evalType, org.apache.tajo.type.Type returnType) { if (!isPrimitiveOpCode(evalType, returnType)) { - throw new CompilationError("No Such OpCode for " + evalType + " returning " + returnType.getType().name()); + throw new CompilationError("No Such OpCode for " + evalType + " returning " + returnType); } - return TUtil.getFromNestedMap(OpCodesMap, evalType, returnType.getType()); + return TUtil.getFromNestedMap(OpCodesMap, evalType, returnType.baseType()); } - public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targetType) { - TajoDataTypes.Type srcRawType = srcType.getType(); - TajoDataTypes.Type targetRawType = targetType.getType(); - switch(srcRawType) { + public void castInsn(org.apache.tajo.type.Type srcType, org.apache.tajo.type.Type targetType) { + TajoDataTypes.Type srcBaseType = srcType.baseType(); + TajoDataTypes.Type targetBaseType = targetType.baseType(); + switch(srcBaseType) { case BOOLEAN: case CHAR: { - if (srcType.hasLength() && srcType.getLength() == 1) { - switch (targetType.getType()) { + Char srcCharType = (Char) srcType; + if (srcCharType.length() == 1) { + switch (targetBaseType) { case CHAR: case INT1: case INT2: @@ -474,7 +479,7 @@ public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targ throw new TajoRuntimeException(new InvalidValueForCastException(srcType, targetType)); } } else { - switch (targetRawType) { + switch (targetBaseType) { case CHAR: case INT1: case INT2: @@ -491,7 +496,7 @@ public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targ case INT1: case INT2: case INT4: - switch (targetType.getType()) { + switch (targetBaseType) { case CHAR: case INT1: methodvisitor.visitInsn(Opcodes.I2C); break; case INT2: methodvisitor.visitInsn(Opcodes.I2S); break; @@ -504,7 +509,7 @@ public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targ } break; case INT8: - switch (targetRawType) { + switch (targetBaseType) { case CHAR: case INT1: case INT2: @@ -517,7 +522,7 @@ public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targ } break; case FLOAT4: - switch (targetRawType) { + switch (targetBaseType) { case CHAR: case INT1: case INT2: @@ -530,7 +535,7 @@ public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targ } break; case FLOAT8: - switch (targetRawType) { + switch (targetBaseType) { case CHAR: case INT1: case INT2: @@ -543,7 +548,7 @@ public void castInsn(TajoDataTypes.DataType srcType, TajoDataTypes.DataType targ } break; case TEXT: - switch (targetRawType) { + switch (targetBaseType) { case CHAR: case INT1: case INT2: @@ -582,7 +587,7 @@ public static String getInternalName(Class clazz) { return clazz.getName().replace('.', '/'); } - public void convertToPrimitive(TajoDataTypes.DataType type) { + public void convertToPrimitive(org.apache.tajo.type.Type type) { Label ifNull = new Label(); Label afterAll = new Label(); @@ -595,7 +600,7 @@ public void convertToPrimitive(TajoDataTypes.DataType type) { methodvisitor.visitJumpInsn(Opcodes.IFEQ, ifNull); // datum aload(datum); - switch (type.getType()) { + switch (type.baseType()) { case BOOLEAN: case INT1: case INT2: @@ -634,11 +639,11 @@ public void convertToPrimitive(TajoDataTypes.DataType type) { methodvisitor.visitLabel(afterAll); } - public void convertToDatum(TajoDataTypes.DataType type, boolean castToDatum) { + public void convertToDatum(org.apache.tajo.type.Type type, boolean castToDatum) { String convertMethod; Class returnType; Class [] paramTypes; - switch (type.getType()) { + switch (type.baseType()) { case NULL_TYPE: pop(); // pop null flag pop(type); // pop null datum @@ -716,7 +721,7 @@ public void convertToDatum(TajoDataTypes.DataType type, boolean castToDatum) { paramTypes = new Class[] {int.class}; break; default: - throw new RuntimeException("Unsupported type: " + type.getType().name()); + throw new RuntimeException("Unsupported type: " + type); } Label ifNull = new Label(); @@ -738,7 +743,7 @@ public void convertToDatum(TajoDataTypes.DataType type, boolean castToDatum) { } } - public void pop(TajoDataTypes.DataType type) { + public void pop(org.apache.tajo.type.Type type) { if (getWordSize(type) == 2) { methodvisitor.visitInsn(Opcodes.POP2); } else { @@ -886,11 +891,11 @@ private int getCurVarIdAndIncrease() { return varId; } - public int store(TajoDataTypes.DataType type) { + public int store(org.apache.tajo.type.Type type) { int varId = nextVarId; nextVarId += TajoGeneratorAdapter.getWordSize(type); - switch (type.getType()) { + switch (type.baseType()) { case NULL_TYPE: case BOOLEAN: case CHAR: @@ -916,14 +921,14 @@ public int store(TajoDataTypes.DataType type) { methodvisitor.visitVarInsn(Opcodes.ASTORE, varId); break; default: - throw new CompilationError("Unknown data type: " + type.getType().name()); + throw new CompilationError("Unknown data type: " + type); } return varId; } - public void emitBoxing(EvalCodeGenContext context, TajoDataTypes.DataType dataType) { - switch (dataType.getType()) { + public void emitBoxing(EvalCodeGenContext context, org.apache.tajo.type.Type type) { + switch (type.baseType()) { case CHAR: case TEXT: @@ -944,12 +949,12 @@ public void emitBoxing(EvalCodeGenContext context, TajoDataTypes.DataType dataTy break; default: - throw new RuntimeException(dataType.getType().name() + " is not supported yet"); + throw new RuntimeException(type + " is not supported yet"); } } - public void emitUnboxing(EvalCodeGenContext context, TajoDataTypes.DataType dataType) { - switch (dataType.getType()) { + public void emitUnboxing(EvalCodeGenContext context, org.apache.tajo.type.Type type) { + switch (type.baseType()) { case CHAR: case TEXT: @@ -970,7 +975,7 @@ public void emitUnboxing(EvalCodeGenContext context, TajoDataTypes.DataType data break; default: - throw new RuntimeException(dataType.getType().name() + " is not supported yet"); + throw new RuntimeException(type + " is not supported yet"); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java index 95ec37105f..9d0f9d0026 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java @@ -55,9 +55,9 @@ public EvalNode visitBinaryEval(EvalCodeGenContext context, Stack stac @Override public EvalNode visitConst(EvalCodeGenContext context, ConstEval constEval, Stack stack) { - if (constEval.getValueType().getType() == TajoDataTypes.Type.INTERVAL) { + if (constEval.getValueType().baseType() == TajoDataTypes.Type.INTERVAL) { if (!context.symbols.containsKey(constEval)) { - String fieldName = constEval.getValueType().getType().name() + "_" + context.seqId++; + String fieldName = constEval.getValueType().baseType().name() + "_" + context.seqId++; context.symbols.put(constEval, fieldName); context.classWriter.visitField(Opcodes.ACC_PRIVATE, fieldName, diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java index 463d0151cd..cdab04f532 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java @@ -349,13 +349,13 @@ private void buildJoinPlanWithUnionChannel(GlobalPlanContext context, JoinNode j private AggregationFunctionCallEval createSumFunction(EvalNode[] args) throws TajoException { FunctionDesc functionDesc = null; functionDesc = getCatalog().getFunction("sum", CatalogProtos.FunctionType.AGGREGATION, - args[0].getValueType()); + TypeConverter.convert(args[0].getValueType())); return new AggregationFunctionCallEval(functionDesc, args); } private AggregationFunctionCallEval createCountFunction(EvalNode [] args) throws TajoException { FunctionDesc functionDesc = getCatalog().getFunction("count", CatalogProtos.FunctionType.AGGREGATION, - args[0].getValueType()); + TypeConverter.convert(args[0].getValueType())); return new AggregationFunctionCallEval(functionDesc, args); } @@ -367,13 +367,13 @@ private AggregationFunctionCallEval createCountRowFunction(EvalNode[] args) thro private AggregationFunctionCallEval createMaxFunction(EvalNode [] args) throws TajoException { FunctionDesc functionDesc = getCatalog().getFunction("max", CatalogProtos.FunctionType.AGGREGATION, - args[0].getValueType()); + TypeConverter.convert(args[0].getValueType())); return new AggregationFunctionCallEval(functionDesc, args); } private AggregationFunctionCallEval createMinFunction(EvalNode [] args) throws TajoException { FunctionDesc functionDesc = getCatalog().getFunction("min", CatalogProtos.FunctionType.AGGREGATION, - args[0].getValueType()); + TypeConverter.convert(args[0].getValueType())); return new AggregationFunctionCallEval(functionDesc, args); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java index d55c4bacf0..660f875c2a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java @@ -32,6 +32,7 @@ import org.apache.tajo.engine.utils.TableCache; import org.apache.tajo.engine.utils.TableCacheKey; import org.apache.tajo.exception.TajoException; +import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.util.Pair; @@ -58,7 +59,6 @@ public void initialize(final QueryContext context, final String planJson) { if (!initializing.getAndSet(true)) { try { ExecutionBlockSharedResource.this.context = context; - initPlan(planJson); initCodeGeneration(); resourceInitSuccess = true; } catch (Throwable t) { @@ -72,10 +72,6 @@ public void initialize(final QueryContext context, final String planJson) { } } - private void initPlan(String planJson) { - plan = CoreGsonHelper.fromJson(planJson, LogicalNode.class); - } - private void initCodeGeneration() throws TajoException { if (context.getBool(SessionVars.CODEGEN)) { codeGenEnabled = true; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index 8107c720ed..e87d33b868 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -23,17 +23,10 @@ import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; import org.apache.tajo.algebra.*; -import org.apache.tajo.catalog.CatalogService; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.exception.UndefinedFunctionException; +import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.*; -import org.apache.tajo.exception.TajoException; -import org.apache.tajo.exception.TajoInternalError; -import org.apache.tajo.exception.NotImplementedException; -import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.exception.*; import org.apache.tajo.plan.algebra.BaseAlgebraVisitor; import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.logical.NodeType; @@ -52,6 +45,7 @@ import static org.apache.tajo.algebra.WindowSpec.WindowFrameEndBoundType; import static org.apache.tajo.algebra.WindowSpec.WindowFrameStartBoundType; +import static org.apache.tajo.catalog.TypeConverter.convert; import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType; import static org.apache.tajo.common.TajoDataTypes.DataType; import static org.apache.tajo.common.TajoDataTypes.Type; @@ -115,8 +109,8 @@ public static void assertEval(boolean condition, String message) throws TajoExce * @return a pair including left/right hand side terms */ private static Pair convertTypesIfNecessary(Context ctx, EvalNode lhs, EvalNode rhs) { - Type lhsType = lhs.getValueType().getType(); - Type rhsType = rhs.getValueType().getType(); + Type lhsType = lhs.getValueType().baseType(); + Type rhsType = rhs.getValueType().baseType(); // If one of both is NULL, it just returns the original types without casting. if (lhsType == Type.NULL_TYPE || rhsType == Type.NULL_TYPE) { @@ -148,11 +142,11 @@ private static Pair convertTypesIfNecessary(Context ctx, Eva private static EvalNode convertType(Context ctx, EvalNode evalNode, DataType toType) { // if original and toType is the same, we don't need type conversion. - if (evalNode.getValueType().equals(toType)) { + if (evalNode.getValueType().equals(TypeConverter.convert(toType))) { return evalNode; } // the conversion to null is not allowed. - if (evalNode.getValueType().getType() == Type.NULL_TYPE || toType.getType() == Type.NULL_TYPE) { + if (evalNode.getValueType().isNull() || toType.getType() == Type.NULL_TYPE) { return evalNode; } @@ -314,7 +308,10 @@ public EvalNode visitBetween(Context ctx, Stack stack, BetweenPredicate be stack.pop(); // implicit type conversion - DataType widestType = CatalogUtil.getWidestType(predicand.getValueType(), begin.getValueType(), end.getValueType()); + DataType widestType = CatalogUtil.getWidestType( + convert(predicand.getValueType()), + convert(begin.getValueType()), + convert(end.getValueType())); BetweenPredicateEval betweenEval = new BetweenPredicateEval( between.isNot(), @@ -343,13 +340,15 @@ public EvalNode visitCaseWhen(Context ctx, Stack stack, CaseWhenPredicate } // Getting the widest type from all if-then expressions and else expression. - DataType widestType = caseWhenEval.getIfThenEvals().get(0).getResult().getValueType(); + DataType widestType = convert(caseWhenEval.getIfThenEvals().get(0).getResult().getValueType()); for (int i = 1; i < caseWhenEval.getIfThenEvals().size(); i++) { - widestType = CatalogUtil.getWidestType(caseWhenEval.getIfThenEvals().get(i).getResult().getValueType(), + widestType = CatalogUtil.getWidestType( + convert(caseWhenEval.getIfThenEvals().get(i).getResult().getValueType()), widestType); } if (caseWhen.hasElseResult()) { - widestType = CatalogUtil.getWidestType(widestType, caseWhenEval.getElse().getValueType()); + widestType = CatalogUtil.getWidestType( + widestType, convert(caseWhenEval.getElse().getValueType())); } assertEval(widestType != null, "Invalid Type Conversion for CaseWhen"); @@ -437,10 +436,10 @@ public EvalNode visitConcatenate(Context ctx, Stack stack, BinaryOperator EvalNode rhs = visit(ctx, stack, expr.getRight()); stack.pop(); - if (lhs.getValueType().getType() != Type.TEXT) { + if (lhs.getValueType().baseType() != Type.TEXT) { lhs = convertType(ctx, lhs, CatalogUtil.newSimpleDataType(Type.TEXT)); } - if (rhs.getValueType().getType() != Type.TEXT) { + if (rhs.getValueType().baseType() != Type.TEXT) { rhs = convertType(ctx, rhs, CatalogUtil.newSimpleDataType(Type.TEXT)); } @@ -585,7 +584,7 @@ public EvalNode visitFunction(Context ctx, Stack stack, FunctionExpr expr) for (int i = 0; i < params.length; i++) { givenArgs[i] = visit(ctx, stack, params[i]); - paramTypes[i] = givenArgs[i].getValueType(); + paramTypes[i] = TypeConverter.convert(givenArgs[i].getValueType()); } stack.pop(); // <--- Pop @@ -662,7 +661,7 @@ public EvalNode visitGeneralSetFunction(Context ctx, Stack stack, GeneralS if (setFunction.getSignature().equalsIgnoreCase("count")) { paramTypes[0] = CatalogUtil.newSimpleDataType(Type.ANY); } else { - paramTypes[0] = givenArgs[0].getValueType(); + paramTypes[0] = TypeConverter.convert(givenArgs[0].getValueType()); } if (!catalog.containFunction(setFunction.getSignature(), functionType, paramTypes)) { @@ -719,11 +718,11 @@ public EvalNode visitWindowFunction(Context ctx, Stack stack, WindowFuncti } else if (windowFunc.getSignature().equalsIgnoreCase("row_number")) { paramTypes[0] = CatalogUtil.newSimpleDataType(Type.INT8); } else { - paramTypes[0] = givenArgs[0].getValueType(); + paramTypes[0] = convert(givenArgs[0].getValueType()); } for (int i = 1; i < params.length; i++) { givenArgs[i] = visit(ctx, stack, params[i]); - paramTypes[i] = givenArgs[i].getValueType(); + paramTypes[i] = convert(givenArgs[i].getValueType()); } } else { if (windowFunc.getSignature().equalsIgnoreCase("rank")) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java index 9176c2f515..f74e16fbba 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java @@ -577,7 +577,7 @@ public static void verifyProjectedFields(QueryBlock block, Projectable projectab public static void prohibitNestedRecordProjection(Projectable projectable) throws TajoException { for (Target t : projectable.getTargets()) { - if (t.getEvalTree().getValueType().getType() == TajoDataTypes.Type.RECORD) { + if (t.getEvalTree().getValueType().isStruct()) { throw new NotImplementedException("record field projection"); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/NamedExprsManager.java b/tajo-plan/src/main/java/org/apache/tajo/plan/NamedExprsManager.java index 6ad9d7068f..53f26105e6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/NamedExprsManager.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/NamedExprsManager.java @@ -34,6 +34,8 @@ import java.util.*; +import static org.apache.tajo.catalog.TypeConverter.convert; + /** * NamedExprsManager manages an expressions used in a query block. All expressions used in a query block must be * added to NamedExprsManager. When an expression is added to NamedExprsManager, NamedExprsManager gives a reference @@ -313,14 +315,14 @@ public Target getTarget(String referenceName, boolean unevaluatedForm) { // But, if this reference name is not primary name, it cannot use the reference name. // It changes the given reference name to the primary name. if (evalNode.getType() != EvalType.CONST && isEvaluated(normalized) && !isPrimaryName(refId, referenceName)) { - return new Target(new FieldEval(getPrimaryName(refId),evalNode.getValueType()), referenceName); + return new Target(new FieldEval(getPrimaryName(refId), convert(evalNode.getValueType())), referenceName); } EvalNode referredEval; if (evalNode.getType() == EvalType.CONST) { referredEval = evalNode; } else { - referredEval = new FieldEval(idToNamesMap.get(refId).get(0), evalNode.getValueType()); + referredEval = new FieldEval(idToNamesMap.get(refId).get(0), convert(evalNode.getValueType())); } return new Target(referredEval, referenceName); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java b/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java index 1ff297f77e..9fc5c0d27d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java @@ -25,18 +25,20 @@ import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.exception.UndefinedFunctionException; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.exception.TajoException; import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.exception.UndefinedFunctionException; import org.apache.tajo.function.FunctionUtil; import org.apache.tajo.plan.nameresolver.NameResolver; import org.apache.tajo.plan.nameresolver.NameResolvingMode; import org.apache.tajo.plan.visitor.SimpleAlgebraVisitor; +import org.apache.tajo.type.Type; import java.util.Stack; +import static org.apache.tajo.catalog.TypeConverter.convert; import static org.apache.tajo.common.TajoDataTypes.DataType; import static org.apache.tajo.common.TajoDataTypes.Type.BOOLEAN; import static org.apache.tajo.common.TajoDataTypes.Type.NULL_TYPE; @@ -81,24 +83,24 @@ public DataType visitBinaryOperator(LogicalPlanner.PlanContext ctx, Stack DataType lhsType = visit(ctx, stack, expr.getLeft()); DataType rhsType = visit(ctx, stack, expr.getRight()); stack.pop(); - return computeBinaryType(expr.getType(), lhsType, rhsType); + return convert(computeBinaryType(expr.getType(), convert(lhsType), convert(rhsType))); } - public DataType computeBinaryType(OpType type, DataType lhsDataType, DataType rhsDataType) throws TajoException { + public Type computeBinaryType(OpType type, Type lhsDataType, Type rhsDataType) throws TajoException { Preconditions.checkNotNull(type); Preconditions.checkNotNull(lhsDataType); Preconditions.checkNotNull(rhsDataType); if(OpType.isLogicalType(type) || OpType.isComparisonType(type)) { - return BOOL_TYPE; + return Type.Bool; } else if (OpType.isArithmeticType(type)) { return DataTypeUtil.determineType(lhsDataType, rhsDataType); } else if (type == OpType.Concatenate) { - return CatalogUtil.newSimpleDataType(TajoDataTypes.Type.TEXT); + return Type.Text; } else if (type == OpType.InPredicate) { - return BOOL_TYPE; + return Type.Bool; } else if (type == OpType.LikePredicate || type == OpType.SimilarToPredicate || type == OpType.Regexp) { - return BOOL_TYPE; + return Type.Bool; } else { throw new TajoInternalError(type.name() + "is not binary type"); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java index 3ff0065050..908496b2a6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java @@ -21,12 +21,13 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes.DataType; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.datum.Datum; import org.apache.tajo.plan.function.AggFunctionInvoke; import org.apache.tajo.plan.function.FunctionContext; import org.apache.tajo.plan.function.FunctionInvokeContext; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; import org.apache.tajo.util.TUtil; import java.io.IOException; @@ -106,11 +107,11 @@ public Datum terminate(FunctionContext context) { } @Override - public DataType getValueType() { + public Type getValueType() { if (!lastPhase) { - return functionInvoke.getPartialResultType(); + return TypeConverter.convert(functionInvoke.getPartialResultType()); } else { - return funcDesc.getReturnType(); + return TypeConverter.convert(funcDesc.getReturnType()); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java index eb3e36bd2f..9c78497412 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java @@ -19,17 +19,14 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; - -import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class BetweenPredicateEval extends EvalNode implements Cloneable { - private static final TajoDataTypes.DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN); @Expose private boolean not; @Expose private boolean symmetric; @Expose private EvalNode predicand; @@ -194,8 +191,8 @@ public Datum eval(Tuple param) { } @Override - public TajoDataTypes.DataType getValueType() { - return RES_TYPE; + public Type getValueType() { + return Type.Bool; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BinaryEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BinaryEval.java index e792c34ff7..b565b80189 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BinaryEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BinaryEval.java @@ -20,21 +20,21 @@ import com.google.common.base.Objects; import com.google.gson.annotations.Expose; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes.DataType; +import org.apache.tajo.DataTypeUtil; +import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.datum.NullDatum; -import org.apache.tajo.DataTypeUtil; import org.apache.tajo.exception.InvalidOperationException; import org.apache.tajo.storage.Tuple; -import static org.apache.tajo.common.TajoDataTypes.Type; +import static org.apache.tajo.type.Type.Bool; +import static org.apache.tajo.type.Type.Text; public class BinaryEval extends EvalNode implements Cloneable { @Expose protected EvalNode leftExpr; @Expose protected EvalNode rightExpr; - @Expose protected DataType returnType; + @Expose protected org.apache.tajo.type.Type returnType; protected BinaryEval(EvalType type) { super(type); @@ -54,7 +54,7 @@ public BinaryEval(EvalType type, EvalNode left, EvalNode right) { type == EvalType.GTH || type == EvalType.LEQ || type == EvalType.GEQ ) { - this.returnType = CatalogUtil.newSimpleDataType(Type.BOOLEAN); + this.returnType = Bool; } else if ( type == EvalType.PLUS || type == EvalType.MINUS || @@ -64,7 +64,7 @@ public BinaryEval(EvalType type, EvalNode left, EvalNode right) { this.returnType = DataTypeUtil.determineType(left.getValueType(), right.getValueType()); } else if (type == EvalType.CONCATENATE) { - this.returnType = CatalogUtil.newSimpleDataType(Type.TEXT); + this.returnType = Text; } } @@ -154,7 +154,7 @@ public Datum eval(Tuple tuple) { return lhs.modular(rhs); case CONCATENATE: - if (lhs.type() == Type.NULL_TYPE || rhs.type() == Type.NULL_TYPE) { + if (lhs.type() == TajoDataTypes.Type.NULL_TYPE || rhs.type() == TajoDataTypes.Type.NULL_TYPE) { return NullDatum.get(); } return DatumFactory.createText(lhs.asChars() + rhs.asChars()); @@ -169,7 +169,7 @@ public String getName() { } @Override - public DataType getValueType() { + public org.apache.tajo.type.Type getValueType() { return returnType; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java index 331b8e138c..d55cff99c3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java @@ -20,11 +20,6 @@ import com.google.common.collect.Lists; import com.google.gson.annotations.Expose; - -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.common.TajoDataTypes.DataType; -import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.json.GsonObject; @@ -68,10 +63,10 @@ public void setElseResult(EvalNode elseResult) { } @Override - public DataType getValueType() { + public org.apache.tajo.type.Type getValueType() { // Find not null type for (IfThenEval eachWhen: whens) { - if (eachWhen.getResult().getValueType().getType() != Type.NULL_TYPE) { + if (!eachWhen.getResult().getValueType().isNull()) { return eachWhen.getResult().getValueType(); } } @@ -80,7 +75,7 @@ public DataType getValueType() { return elseResult.getValueType(); } - return NullDatum.getDataType(); + return org.apache.tajo.type.Type.Null; } @Override @@ -202,8 +197,8 @@ public IfThenEval(EvalNode condition, EvalNode result) { } @Override - public DataType getValueType() { - return CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN); + public org.apache.tajo.type.Type getValueType() { + return org.apache.tajo.type.Type.Bool; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java index b8b768bd0c..653eeb0ce1 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java @@ -19,12 +19,13 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; - import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; import org.apache.tajo.util.TUtil; import java.util.TimeZone; @@ -50,8 +51,8 @@ public EvalNode getOperand() { } @Override - public DataType getValueType() { - return target; + public Type getValueType() { + return TypeConverter.convert(target); } public boolean hasTimeZone() { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/ConstEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/ConstEval.java index ff964e631f..25c12d4fd6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/ConstEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/ConstEval.java @@ -20,10 +20,10 @@ import com.google.common.base.Objects; import com.google.gson.annotations.Expose; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes.DataType; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class ConstEval extends EvalNode implements Comparable, Cloneable { @Expose Datum datum = null; @@ -50,8 +50,8 @@ public Datum eval(Tuple tuple) { } @Override - public DataType getValueType() { - return CatalogUtil.newSimpleDataType(datum.type()); + public Type getValueType() { + return TypeConverter.convert(datum.type()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java index b154532f12..665a770d18 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java @@ -29,6 +29,7 @@ import org.apache.tajo.plan.serder.PlanGsonHelper; import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; /** * An annotated expression which includes actual data domains. @@ -50,7 +51,7 @@ public EvalType getType() { return this.type; } - public abstract DataType getValueType(); + public abstract Type getValueType(); public abstract int childNum(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java index c71dba282e..bb4803048a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java @@ -179,7 +179,7 @@ public static DataType getDomainByExpr(Schema inputSchema, EvalNode expr) { case DIVIDE: case CONST: case FUNCTION: - return expr.getValueType(); + return TypeConverter.convert(expr.getValueType()); case FIELD: FieldEval fieldEval = (FieldEval) expr; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java index 2b301b6465..60144cea0e 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java @@ -21,9 +21,11 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class FieldEval extends EvalNode implements Cloneable { @Expose private Column column; @@ -33,6 +35,11 @@ public FieldEval(String columnName, DataType domain) { super(EvalType.FIELD); this.column = new Column(columnName, domain); } + + public FieldEval(String columnName, Type type) { + super(EvalType.FIELD); + this.column = new Column(columnName, TypeConverter.convert(type)); + } public FieldEval(Column column) { super(EvalType.FIELD); @@ -62,8 +69,8 @@ public Datum eval(Tuple tuple) { } @Override - public DataType getValueType() { - return column.getDataType(); + public Type getValueType() { + return TypeConverter.convert(column.getDataType()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java index e4bb46cffe..1a4ea8e3d2 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java @@ -23,11 +23,13 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; +import org.apache.tajo.type.Type; import org.apache.tajo.util.TUtil; import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType.DISTINCT_AGGREGATION; @@ -71,7 +73,7 @@ public FunctionDesc getFuncDesc() { ParamType [] paramTypes = new ParamType[argEvals.length]; for (int i = 0; i < argEvals.length; i++) { if (argEvals[i].getType() == EvalType.CONST) { - if (argEvals[i].getValueType().getType() == TajoDataTypes.Type.NULL_TYPE) { + if (argEvals[i].getValueType().isNull()) { paramTypes[i] = ParamType.NULL; } else { paramTypes[i] = ParamType.CONSTANT; @@ -111,8 +113,8 @@ public EvalNode getChild(int idx) { } - public DataType getValueType() { - return this.funcDesc.getReturnType(); + public Type getValueType() { + return TypeConverter.convert(this.funcDesc.getReturnType()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java index 66bb5b0425..5ee6e9ef2b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/InEval.java @@ -26,14 +26,13 @@ import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; import java.util.Arrays; import java.util.HashSet; import java.util.Set; public class InEval extends BinaryEval { - private static final TajoDataTypes.DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN); - @Expose private boolean not; Set values; @@ -47,8 +46,8 @@ public boolean isNot() { } @Override - public TajoDataTypes.DataType getValueType() { - return RES_TYPE; + public Type getValueType() { + return Type.Bool; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/IsNullEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/IsNullEval.java index 72b36ebc44..f337b8a0e6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/IsNullEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/IsNullEval.java @@ -19,19 +19,12 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; - -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class IsNullEval extends UnaryEval { - // it's just a hack to emulate a binary expression - private final static ConstEval DUMMY_EVAL = new ConstEval(DatumFactory.createBool(true)); - private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN); - // persistent variables @Expose private boolean isNot; @@ -41,8 +34,8 @@ public IsNullEval(boolean not, EvalNode predicand) { } @Override - public DataType getValueType() { - return RES_TYPE; + public Type getValueType() { + return Type.Bool; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/NotEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/NotEval.java index 088f3a5632..f9b5393f15 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/NotEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/NotEval.java @@ -18,23 +18,19 @@ package org.apache.tajo.plan.expr; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class NotEval extends UnaryEval implements Cloneable { - private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN); - public NotEval(EvalNode child) { super(EvalType.NOT, child); } @Override - public DataType getValueType() { - return RES_TYPE; + public Type getValueType() { + return Type.Bool; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PartialBinaryExpr.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PartialBinaryExpr.java index 1b1ee448ac..78b89102bc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PartialBinaryExpr.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PartialBinaryExpr.java @@ -18,10 +18,10 @@ package org.apache.tajo.plan.expr; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.exception.InvalidOperationException; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class PartialBinaryExpr extends BinaryEval { @@ -37,7 +37,7 @@ public PartialBinaryExpr(EvalType type, EvalNode left, EvalNode right) { } @Override - public DataType getValueType() { + public Type getValueType() { return null; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java index ec143f7ed2..0f595b1657 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java @@ -19,21 +19,17 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; -import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; public abstract class PatternMatchPredicateEval extends BinaryEval { - private static final DataType RES_TYPE = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.BOOLEAN); - @Expose protected boolean not; @Expose protected String pattern; @Expose protected boolean caseInsensitive; @@ -64,8 +60,8 @@ public boolean isCaseInsensitive() { } @Override - public DataType getValueType() { - return RES_TYPE; + public Type getValueType() { + return Type.Bool; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/RowConstantEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/RowConstantEval.java index d2dae5a4e1..aaaba22ee3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/RowConstantEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/RowConstantEval.java @@ -19,17 +19,16 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; -import org.apache.tajo.catalog.CatalogUtil; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; import org.apache.tajo.util.StringUtils; import org.apache.tajo.util.TUtil; import java.util.Arrays; -import static org.apache.tajo.common.TajoDataTypes.DataType; - public class RowConstantEval extends ValueSetEval { @Expose Datum [] values; @@ -39,8 +38,8 @@ public RowConstantEval(Datum [] values) { } @Override - public DataType getValueType() { - return CatalogUtil.newSimpleDataType(values[0].type()); + public Type getValueType() { + return TypeConverter.convert(values[0].type()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SignedEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SignedEval.java index baa78a2bb5..085d82c841 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SignedEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SignedEval.java @@ -20,10 +20,10 @@ import com.google.common.base.Objects; import com.google.gson.annotations.Expose; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.NumericDatum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public class SignedEval extends UnaryEval implements Cloneable { @Expose private boolean negative; @@ -38,7 +38,7 @@ public boolean isNegative() { } @Override - public DataType getValueType() { + public Type getValueType() { return child.getValueType(); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SubqueryEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SubqueryEval.java index f2af073c99..1cac7ffa15 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SubqueryEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SubqueryEval.java @@ -20,12 +20,13 @@ import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes.DataType; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.datum.Datum; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.plan.logical.TableSubQueryNode; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; /** * SubqueryEval is a temporal eval to keep subquery information when the subquery occurs in expressions, @@ -44,8 +45,8 @@ public SubqueryEval(TableSubQueryNode subQueryNode) { } @Override - public DataType getValueType() { - return subQueryNode.getOutSchema().getColumn(0).getDataType(); + public Type getValueType() { + return TypeConverter.convert(subQueryNode.getOutSchema().getColumn(0).getDataType()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/UnaryEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/UnaryEval.java index 4db2b90d6e..5c7ee6e73d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/UnaryEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/UnaryEval.java @@ -21,9 +21,9 @@ import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.gson.annotations.Expose; -import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; public abstract class UnaryEval extends EvalNode implements Cloneable { @Expose protected EvalNode child; @@ -56,7 +56,7 @@ public EvalNode getChild() { } @Override - public TajoDataTypes.DataType getValueType() { + public Type getValueType() { return null; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java index e5b88f2456..6c703951c3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/WindowFunctionEval.java @@ -21,11 +21,13 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.SortSpec; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.plan.function.FunctionContext; import org.apache.tajo.plan.logical.WindowSpec; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.type.Type; import org.apache.tajo.util.StringUtils; import org.apache.tajo.util.TUtil; @@ -71,8 +73,8 @@ public Datum terminate(FunctionContext context) { } @Override - public DataType getValueType() { - return funcDesc.getReturnType(); + public Type getValueType() { + return TypeConverter.convert(funcDesc.getReturnType()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java index f37ce69587..759867f37e 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java @@ -23,6 +23,7 @@ import com.google.protobuf.ByteString; import org.apache.tajo.algebra.WindowSpec.WindowFrameEndBoundType; import org.apache.tajo.algebra.WindowSpec.WindowFrameStartBoundType; +import org.apache.tajo.catalog.TypeConverter; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.datum.AnyDatum; import org.apache.tajo.datum.Datum; @@ -96,7 +97,7 @@ private PlanProto.EvalNode.Builder createEvalBuilder(EvalTreeProtoBuilderContext PlanProto.EvalNode.Builder nodeBuilder = PlanProto.EvalNode.newBuilder(); nodeBuilder.setId(sid); - nodeBuilder.setDataType(node.getValueType()); + nodeBuilder.setDataType(TypeConverter.convert(node.getValueType())); nodeBuilder.setType(PlanProto.EvalType.valueOf(node.getType().name())); return nodeBuilder; } @@ -118,7 +119,7 @@ public EvalNode visitUnaryEval(EvalTreeProtoBuilderContext context, UnaryEval un unaryBuilder.setNegative(signedEval.isNegative()); } else if (unary.getType() == EvalType.CAST) { CastEval castEval = (CastEval) unary; - unaryBuilder.setCastingType(castEval.getValueType()); + unaryBuilder.setCastingType(TypeConverter.convert(castEval.getValueType())); if (castEval.hasTimeZone()) { unaryBuilder.setTimezone(castEval.getTimezone().getID()); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/EvalNodeToExprConverter.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/EvalNodeToExprConverter.java index 00638c2f66..86f82860e6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/EvalNodeToExprConverter.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/EvalNodeToExprConverter.java @@ -145,7 +145,7 @@ protected EvalNode visitBinaryEval(Object o, Stack stack, BinaryEval b @Override protected EvalNode visitConst(Object o, ConstEval evalNode, Stack stack) { - exprs.push(convertDatumToExpr(evalNode.getValueType().getType(), evalNode.getValue())); + exprs.push(convertDatumToExpr(evalNode.getValueType().baseType(), evalNode.getValue())); return super.visitConst(o, evalNode, stack); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java index ff3e3c8f9b..7dc5ae3109 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java @@ -25,8 +25,6 @@ import org.apache.tajo.algebra.*; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.*; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.exception.*; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.Target; @@ -36,6 +34,7 @@ import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor; import org.apache.tajo.plan.visitor.ExplainLogicalPlanVisitor; import org.apache.tajo.plan.visitor.SimpleAlgebraVisitor; +import org.apache.tajo.type.Type; import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.util.StringUtils; @@ -658,10 +657,10 @@ public static Schema targetToSchema(Collection targets) { public static Schema targetToSchema(List targets) { SchemaBuilder schema = SchemaBuilder.uniqueNameBuilder(); for (Target t : targets) { - DataType type = t.getEvalTree().getValueType(); + Type type = t.getEvalTree().getValueType(); // hack to avoid projecting record type. - if (type.getType() == TajoDataTypes.Type.RECORD) { + if (type.isStruct()) { throw new TajoRuntimeException(new NotImplementedException("record projection")); } @@ -672,7 +671,7 @@ public static Schema targetToSchema(List targets) { name = t.getEvalTree().getName(); } - schema.add(name, type); + schema.add(name, TypeConverter.convert(type)); } return schema.build(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/ExprsVerifier.java b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/ExprsVerifier.java index a9737e0cc1..2040ae3632 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/ExprsVerifier.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/ExprsVerifier.java @@ -19,6 +19,7 @@ package org.apache.tajo.plan.verifier; import org.apache.tajo.catalog.Column; +import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.error.Errors; import org.apache.tajo.exception.TajoException; import org.apache.tajo.exception.TajoInternalError; @@ -29,8 +30,7 @@ import java.util.Set; import java.util.Stack; -import static org.apache.tajo.common.TajoDataTypes.DataType; -import static org.apache.tajo.common.TajoDataTypes.Type; +import static org.apache.tajo.common.TajoDataTypes.Type.*; /** * It verifies one predicate or expression with the semantic and data type checks as follows: @@ -61,7 +61,7 @@ public static VerificationState verify(VerificationState state, LogicalNode curr /** * It checks the compatibility of two data types. */ - private static boolean isCompatibleType(DataType dataType1, DataType dataType2) { + private static boolean isCompatibleType(org.apache.tajo.type.Type dataType1, org.apache.tajo.type.Type dataType2) { if (checkNumericType(dataType1) && checkNumericType(dataType2)) { return true; } @@ -85,9 +85,7 @@ private static boolean isCompatibleType(DataType dataType1, DataType dataType2) * It checks both expressions in a comparison operator are compatible to each other. */ private static void verifyComparisonOperator(VerificationState state, BinaryEval expr) { - DataType leftType = expr.getLeftExpr().getValueType(); - DataType rightType = expr.getRightExpr().getValueType(); - if (!isCompatibleType(leftType, rightType)) { + if (!isCompatibleType(expr.getLeftExpr().getValueType(), expr.getRightExpr().getValueType())) { state.addVerification(new UndefinedOperatorException(expr.toString())); } } @@ -145,32 +143,32 @@ private static void checkArithmeticOperand(VerificationState state, BinaryEval e EvalNode leftExpr = evalNode.getLeftExpr(); EvalNode rightExpr = evalNode.getRightExpr(); - DataType leftDataType = leftExpr.getValueType(); - DataType rightDataType = rightExpr.getValueType(); + org.apache.tajo.type.Type leftDataType = leftExpr.getValueType(); + org.apache.tajo.type.Type rightDataType = rightExpr.getValueType(); - Type leftType = leftDataType.getType(); - Type rightType = rightDataType.getType(); + TajoDataTypes.Type leftType = leftDataType.baseType(); + TajoDataTypes.Type rightType = rightDataType.baseType(); - if (leftType == Type.DATE && + if (leftType == DATE && (checkIntType(rightDataType) || - rightType == Type.DATE || rightType == Type.INTERVAL || rightType == Type.TIME)) { + rightType == DATE || rightType == INTERVAL || rightType == TIME)) { return; } - if (leftType == Type.INTERVAL && + if (leftType == INTERVAL && (checkNumericType(rightDataType) || - rightType == Type.DATE || rightType == Type.INTERVAL || rightType == Type.TIME || - rightType == Type.TIMESTAMP)) { + rightType == DATE || rightType == INTERVAL || rightType == TIME || + rightType == TIMESTAMP)) { return; } - if (leftType == Type.TIME && - (rightType == Type.DATE || rightType == Type.INTERVAL || rightType == Type.TIME)) { + if (leftType == TIME && + (rightType == DATE || rightType == INTERVAL || rightType == TIME)) { return; } - if (leftType == Type.TIMESTAMP && - (rightType == Type.TIMESTAMP || rightType == Type.INTERVAL || rightType == Type.TIME)) { + if (leftType == TIMESTAMP && + (rightType == TIMESTAMP || rightType == INTERVAL || rightType == TajoDataTypes.Type.TIME)) { return; } @@ -179,29 +177,29 @@ private static void checkArithmeticOperand(VerificationState state, BinaryEval e } } - private static boolean checkNetworkType(DataType dataType) { - return dataType.getType() == Type.INET4 || dataType.getType() == Type.INET6; + private static boolean checkNetworkType(org.apache.tajo.type.Type dataType) { + return dataType.baseType() == INET4 || dataType.baseType() == INET6; } - private static boolean checkIntType(DataType dataType) { - int typeNumber = dataType.getType().getNumber(); - return Type.INT1.getNumber() < typeNumber && typeNumber <= Type.INT8.getNumber(); + private static boolean checkIntType(org.apache.tajo.type.Type dataType) { + int typeNumber = dataType.baseType().getNumber(); + return INT1.getNumber() < typeNumber && typeNumber <= INT8.getNumber(); } - private static boolean checkNumericType(DataType dataType) { - int typeNumber = dataType.getType().getNumber(); - return Type.INT1.getNumber() <= typeNumber && typeNumber <= Type.NUMERIC.getNumber(); + private static boolean checkNumericType(org.apache.tajo.type.Type dataType) { + int typeNumber = dataType.baseType().getNumber(); + return INT1.getNumber() <= typeNumber && typeNumber <= NUMERIC.getNumber(); } - private static boolean checkTextData(DataType dataType) { - int typeNumber = dataType.getType().getNumber(); - return Type.CHAR.getNumber() <= typeNumber && typeNumber <= Type.TEXT.getNumber(); + private static boolean checkTextData(org.apache.tajo.type.Type dataType) { + int typeNumber = dataType.baseType().getNumber(); + return CHAR.getNumber() <= typeNumber && typeNumber <= TEXT.getNumber(); } - private static boolean checkDateTime(DataType dataType) { - int typeNumber = dataType.getType().getNumber(); - return (Type.DATE.getNumber() <= typeNumber && typeNumber <= Type.INTERVAL.getNumber()) || - (Type.TIMEZ.getNumber() <= typeNumber && typeNumber <= Type.TIMESTAMPZ.getNumber()); + private static boolean checkDateTime(org.apache.tajo.type.Type dataType) { + int typeNumber = dataType.baseType().getNumber(); + return (DATE.getNumber() <= typeNumber && typeNumber <= INTERVAL.getNumber()) || + (TIMEZ.getNumber() <= typeNumber && typeNumber <= TIMESTAMPZ.getNumber()); } @Override diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java index a265efe4b5..0b1df31054 100644 --- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java @@ -26,6 +26,7 @@ import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnsupportedDataTypeException; import org.apache.tajo.plan.expr.*; +import org.apache.tajo.type.Type; import org.apache.tajo.util.StringUtils; import java.sql.DatabaseMetaData; @@ -270,8 +271,8 @@ public String convertDatumToSQLLiteral(Datum d) { * @param dataType Tajo DataType * @return SQL DataType */ - public String convertTajoTypeToSQLType(DataType dataType) { - switch (dataType.getType()) { + public String convertTajoTypeToSQLType(Type dataType) { + switch (dataType.baseType()) { case INT1: return "TINYINT"; case INT2: @@ -285,7 +286,7 @@ public String convertTajoTypeToSQLType(DataType dataType) { case FLOAT8: return "DOUBLE"; default: - return dataType.getType().name(); + return dataType.baseType().name(); } } From 71193b218f55d33fad0f255386165b8c88bb310e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 Apr 2016 21:12:58 +0900 Subject: [PATCH 08/20] TAJO-2126: Allow parallel execution of non-leaf sibling ExecutionBlocks. Closes #1003 --- CHANGES | 3 +++ .../planner/global/ParallelExecutionQueue.java | 13 ++----------- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/CHANGES b/CHANGES index 7ac7d832e9..38aeddabf6 100644 --- a/CHANGES +++ b/CHANGES @@ -10,6 +10,9 @@ Release 0.12.0 - unreleased IMPROVEMENT + TAJO-2126: Allow parallel execution of non-leaf sibling ExecutionBlocks. + (jihooon) + TAJO-2127: Use Type instead of DataType for EvalNode. (hyunsik) TAJO-2125: Detect JAVA_HOME on OSX, if JAVA_HOME is not set. (jinho) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ParallelExecutionQueue.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ParallelExecutionQueue.java index 23e309f502..d197bda7ef 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ParallelExecutionQueue.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ParallelExecutionQueue.java @@ -101,17 +101,8 @@ public synchronized ExecutionBlock[] next(ExecutionBlockId doneNow) { } private boolean isExecutableNow(ExecutionBlock current) { - ExecutionBlock parent = masterPlan.getParent(current); - - List dependents = masterPlan.getChilds(current); - if (parent != null && masterPlan.getChannel(current.getId(), parent.getId()).needShuffle()) { - // add all children of sibling for partitioning - dependents = new ArrayList<>(); - for (ExecutionBlock sibling : masterPlan.getChilds(parent)) { - dependents.addAll(masterPlan.getChilds(sibling)); - } - } - for (ExecutionBlock child : dependents) { + + for (ExecutionBlock child : masterPlan.getChilds(current)) { if (!executed.contains(child.getId())) { return false; // there's something should be done before this } From fa80b4c7e22510c4af11d804a812e2c78910ce55 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 22 Apr 2016 22:11:02 +0900 Subject: [PATCH 09/20] trigger ci --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9a71e8e66b..522ed6fe3b 100644 --- a/pom.xml +++ b/pom.xml @@ -92,13 +92,13 @@ tajo-sql-parser tajo-storage tajo-pullserver + tajo-yarn tajo-dist tajo-thirdparty/asm tajo-cli tajo-metrics tajo-core-tests tajo-cluster-tests - tajo-yarn From a1fcc45e737113130054e747da1c823dca8e76dc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 Apr 2016 12:18:23 +0900 Subject: [PATCH 10/20] Cleanup dependency --- pom.xml | 2 +- .../org/apache/tajo/exception/ErrorUtil.java | 14 ++- .../org/apache/tajo/worker/LocalFetcher.java | 46 ++++---- .../org/apache/tajo/worker/RemoteFetcher.java | 1 - tajo-yarn/pom.xml | 110 +++++++++++++++--- 5 files changed, 130 insertions(+), 43 deletions(-) diff --git a/pom.xml b/pom.xml index 522ed6fe3b..71c062bbfa 100644 --- a/pom.xml +++ b/pom.xml @@ -93,12 +93,12 @@ tajo-storage tajo-pullserver tajo-yarn - tajo-dist tajo-thirdparty/asm tajo-cli tajo-metrics tajo-core-tests tajo-cluster-tests + tajo-dist diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/ErrorUtil.java b/tajo-common/src/main/java/org/apache/tajo/exception/ErrorUtil.java index 9a71bd69eb..957b3d1bc8 100644 --- a/tajo-common/src/main/java/org/apache/tajo/exception/ErrorUtil.java +++ b/tajo-common/src/main/java/org/apache/tajo/exception/ErrorUtil.java @@ -34,12 +34,14 @@ public static boolean isFailed(ResultCode code) { public static Stacktrace.StackTrace convertStacktrace(Throwable t) { Stacktrace.StackTrace.Builder builder = Stacktrace.StackTrace.newBuilder(); - for (StackTraceElement element : t.getStackTrace()) { - builder.addElement(Stacktrace.StackTrace.Element.newBuilder() - .setFilename(element.getFileName() == null ? "(Unknown Source)" : element.getFileName()) - .setFunction(element.getClassName() + "::" + element.getMethodName()) - .setLine(element.getLineNumber()) - ); + if (t != null) { + for (StackTraceElement element : t.getStackTrace()) { + builder.addElement(Stacktrace.StackTrace.Element.newBuilder() + .setFilename(element.getFileName() == null ? "(Unknown Source)" : element.getFileName()) + .setFunction(element.getClassName() + "::" + element.getMethodName()) + .setLine(element.getLineNumber()) + ); + } } return builder.build(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java index e362a38c4f..4cf7307797 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java @@ -276,18 +276,23 @@ private List getChunksForRangeShuffle(final PullServerParams params, // Wait for the server to close the connection. throw exception if failed channel.closeFuture().syncUninterruptibly(); - state = FetcherState.FETCH_DATA_FETCHING; - for (FileChunkMeta eachMeta : chunkMetas) { - Path outputPath = StorageUtil.concatPath(queryBaseDir, eachMeta.getTaskId(), "output"); - if (!localDirAllocator.ifExists(outputPath.toString(), conf)) { - LOG.warn("Range shuffle - file not exist. " + outputPath); - continue; + if (!state.equals(FetcherState.FETCH_META_FINISHED)) { + endFetch(FetcherState.FETCH_FAILED); + } else { + state = FetcherState.FETCH_DATA_FETCHING; + for (FileChunkMeta eachMeta : chunkMetas) { + Path outputPath = StorageUtil.concatPath(queryBaseDir, eachMeta.getTaskId(), "output"); + if (!localDirAllocator.ifExists(outputPath.toString(), conf)) { + LOG.warn("Range shuffle - file not exist. " + outputPath); + continue; + } + Path path = localFileSystem.makeQualified(localDirAllocator.getLocalPathToRead(outputPath.toString(), conf)); + FileChunk chunk = new FileChunk(new File(URI.create(path.toUri() + "/output")), + eachMeta.getStartOffset(), eachMeta.getLength()); + chunk.setEbId(tableName); + fileChunks.add(chunk); } - Path path = localFileSystem.makeQualified(localDirAllocator.getLocalPathToRead(outputPath.toString(), conf)); - FileChunk chunk = new FileChunk(new File(URI.create(path.toUri() + "/output")), - eachMeta.getStartOffset(), eachMeta.getLength()); - chunk.setEbId(tableName); - fileChunks.add(chunk); + endFetch(FetcherState.FETCH_DATA_FINISHED); } return fileChunks; @@ -296,8 +301,6 @@ private List getChunksForRangeShuffle(final PullServerParams params, // Close the channel to exit. future.channel().close().awaitUninterruptibly(); } - - endFetch(FetcherState.FETCH_DATA_FINISHED); } } @@ -364,12 +367,14 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) try { if (content.isReadable()) { int contentLength = content.readableBytes(); - if ((totalReceivedContentLength + contentLength) == length) { - state = FetcherState.FETCH_META_FINISHED; - } content.readBytes(buf, totalReceivedContentLength, contentLength); totalReceivedContentLength += contentLength; - if (state.equals(FetcherState.FETCH_META_FINISHED)) { + } + + if (msg instanceof LastHttpContent) { + if (totalReceivedContentLength == length) { + state = FetcherState.FETCH_META_FINISHED; + List jsonMetas = gson.fromJson(new String(buf), List.class); for (String eachJson : jsonMetas) { FileChunkMeta meta = gson.fromJson(eachJson, FileChunkMeta.class); @@ -377,9 +382,10 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) } totalReceivedContentLength = 0; length = -1; - } else if (totalReceivedContentLength > length) { - // TODO - throw new IOException("Illegal length: " + totalReceivedContentLength + ", expected length: " + length); + } else { + endFetch(FetcherState.FETCH_FAILED); + throw new IOException("Invalid fetch meta length: " + totalReceivedContentLength + ", expected length: " + + length); } } } catch (Exception e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java index fa87b9bd9d..9328c977de 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java @@ -240,7 +240,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) content.readBytes(fc, content.readableBytes()); } - // TODO: check compatability with yarn pull server if (msg instanceof LastHttpContent) { if (raf != null) { fileLen = file.length(); diff --git a/tajo-yarn/pom.xml b/tajo-yarn/pom.xml index 12061bfc7d..70511a187b 100644 --- a/tajo-yarn/pom.xml +++ b/tajo-yarn/pom.xml @@ -1,20 +1,20 @@ org.apache.tajo tajo-common + + + org.iq80.snappy + snappy + + + commons-logging + commons-logging-api + + + io.netty + netty-buffer + + org.apache.tajo tajo-storage-common + + + net.minidev + json-smart + + + org.apache.tajo + tajo-plan + + + io.netty + netty-buffer + + org.apache.tajo tajo-storage-hdfs + + + org.apache.hive + hive-storage-api + + + org.apache.hive + hive-orc + + + org.apache.parquet + parquet-hadoop-bundle + + + org.apache.avro + trevni-avro + + + org.apache.avro + trevni-core + + + io.netty + netty-codec-http + + + io.netty + netty-codec + + + io.netty + netty-transport + + + io.netty + netty-buffer + + org.apache.tajo tajo-pullserver + + + org.apache.tajo + tajo-rpc-protobuf + + + io.netty + netty-transport + + + io.netty + netty-codec-http + + From 676439c131aa346226ed5abeb245db95d79c6803 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 25 Apr 2016 17:58:15 +0900 Subject: [PATCH 11/20] Add error handling --- .../main/java/org/apache/tajo/yarn/FadvisedFileRegion.java | 6 ++++-- .../java/org/apache/tajo/yarn/TajoPullServerService.java | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java index ab2414de67..17a753fa22 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java @@ -46,7 +46,7 @@ public class FadvisedFileRegion extends DefaultFileRegion { private final long position; private final int shuffleBufferSize; private final boolean shuffleTransferToAllowed; - private final FileChannel fileChannel; + private FileChannel fileChannel; private ReadaheadPool.ReadaheadRequest readaheadRequest; public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024; @@ -150,7 +150,9 @@ long customShuffleTransfer(WritableByteChannel target, long position) public void releaseExternalResources() { if (readaheadRequest != null) { readaheadRequest.cancel(); + readaheadRequest = null; } + fileChannel = null; super.releaseExternalResources(); } @@ -159,7 +161,7 @@ public void releaseExternalResources() { * we don't need the region to be cached anymore. */ public void transferSuccessful() { - if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0) { + if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null) { try { PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), NativeIO.POSIX.POSIX_FADV_DONTNEED); diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java index 91ab7b5126..369e17f193 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/TajoPullServerService.java @@ -543,8 +543,10 @@ private ChannelFuture sendFile(ChannelHandlerContext ctx, LOG.fatal(file.getFile() + " not found"); return null; } catch (Throwable e) { + LOG.fatal("error while sending a file: ", e); if (spill != null) { //should close a opening file + LOG.warn("Close the file " + file.getFile().getAbsolutePath()); spill.close(); } return null; From c3bddc91a1ff19bd02bce33d401653c7664433f3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 11:18:50 +0900 Subject: [PATCH 12/20] Commons-lang version upgrade --- tajo-pullserver/pom.xml | 6 ++++++ .../org/apache/tajo/pullserver/PullServerUtil.java | 11 ++++++----- .../java/org/apache/tajo/yarn/FadvisedFileRegion.java | 5 +++-- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/tajo-pullserver/pom.xml b/tajo-pullserver/pom.xml index 6c805ac09c..0ddf0133d3 100644 --- a/tajo-pullserver/pom.xml +++ b/tajo-pullserver/pom.xml @@ -46,6 +46,12 @@ + + org.apache.commons + commons-lang3 + 3.4 + + io.netty netty-transport diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java index 4b69db99f7..7b50c0c994 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java @@ -23,7 +23,7 @@ import com.google.gson.Gson; import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.commons.codec.binary.Base64; -import org.apache.commons.lang.reflect.MethodUtils; +import org.apache.commons.lang3.reflect.MethodUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; @@ -46,10 +46,7 @@ import java.lang.reflect.Method; import java.net.URI; import java.net.URLEncoder; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; +import java.util.*; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -112,6 +109,10 @@ private static boolean loadNativeIO() { if (posixFadviseIfPossible == null) { loaded = false; + } else { + final StringBuilder sb = new StringBuilder("posixFadviseIfPossible: "); + Arrays.stream(posixFadviseIfPossible.getParameterTypes()).forEach(p -> sb.append(p.getCanonicalName()).append(", ")); + LOG.info(sb); } return loaded; } diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java index 17a753fa22..e237654885 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java @@ -161,12 +161,13 @@ public void releaseExternalResources() { * we don't need the region to be cached anymore. */ public void transferSuccessful() { - if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null) { + final boolean channelOpen = fileChannel.isOpen(); + if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null && channelOpen) { try { PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), NativeIO.POSIX.POSIX_FADV_DONTNEED); } catch (Throwable t) { - LOG.warn("Failed to manage OS cache for " + identifier, t); + LOG.warn("Failed to manage OS cache for " + identifier + ", valid fd? " + fd.valid() + ", pos: " + getPosition() + ", count: " + getCount() + ", channelOpen: " + channelOpen, t); } } } From a835e498812853dbafc53d1bc6136ada4ca7a0c3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 11:56:30 +0900 Subject: [PATCH 13/20] Keep CacheManipulator instance --- .../org/apache/tajo/pullserver/PullServerUtil.java | 10 ++++++---- .../java/org/apache/tajo/yarn/FadvisedFileRegion.java | 1 + 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java index 7b50c0c994..b12f175cc3 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java @@ -56,6 +56,7 @@ public class PullServerUtil { private static boolean nativeIOPossible = false; private static Method posixFadviseIfPossible; + private static Object cacheManipulator; static { if (NativeIO.isAvailable() && loadNativeIO()) { @@ -78,10 +79,10 @@ public static void posixFadviseIfPossible(String identifier, java.io.FileDescrip long offset, long len, int flags) { if (nativeIOPossible) { try { - posixFadviseIfPossible.invoke(null, identifier, fd, offset, len, flags); + posixFadviseIfPossible.invoke(cacheManipulator, identifier, fd, offset, len, flags); } catch (Throwable t) { nativeIOPossible = false; - LOG.warn("Failed to manage OS cache for " + identifier, t); + LOG.warn("Failed to manage OS cache for " + identifier + ", fd: " + fd.valid() + ", offset: " + offset + ", len: " + len + ", flags: " + flags + ", msg: " + t.getMessage(), t); } } } @@ -96,9 +97,10 @@ private static boolean loadNativeIO() { Method getCacheManipulator = MethodUtils.getAccessibleMethod(NativeIO.POSIX.class, "getCacheManipulator", new Class[0]); Class posixClass; if (getCacheManipulator != null) { - Object posix = MethodUtils.invokeStaticMethod(NativeIO.POSIX.class, "getCacheManipulator", null); - posixClass = posix.getClass(); + cacheManipulator = MethodUtils.invokeStaticMethod(NativeIO.POSIX.class, "getCacheManipulator", null); + posixClass = cacheManipulator.getClass(); } else { + cacheManipulator = null; posixClass = NativeIO.POSIX.class; } posixFadviseIfPossible = MethodUtils.getAccessibleMethod(posixClass, "posixFadviseIfPossible", parameters); diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java index e237654885..503f8b8ee0 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java @@ -165,6 +165,7 @@ public void transferSuccessful() { if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null && channelOpen) { try { PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), +// NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), NativeIO.POSIX.POSIX_FADV_DONTNEED); } catch (Throwable t) { LOG.warn("Failed to manage OS cache for " + identifier + ", valid fd? " + fd.valid() + ", pos: " + getPosition() + ", count: " + getCount() + ", channelOpen: " + channelOpen, t); From ae4d4256c2b59d83a118c042575b01597c017723 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 12:06:18 +0900 Subject: [PATCH 14/20] cleanup --- tajo-pullserver/pom.xml | 6 ------ .../java/org/apache/tajo/pullserver/PullServerUtil.java | 8 ++------ .../java/org/apache/tajo/yarn/FadvisedFileRegion.java | 3 +-- 3 files changed, 3 insertions(+), 14 deletions(-) diff --git a/tajo-pullserver/pom.xml b/tajo-pullserver/pom.xml index 0ddf0133d3..6c805ac09c 100644 --- a/tajo-pullserver/pom.xml +++ b/tajo-pullserver/pom.xml @@ -46,12 +46,6 @@ - - org.apache.commons - commons-lang3 - 3.4 - - io.netty netty-transport diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java index b12f175cc3..9fcea39160 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java @@ -23,7 +23,7 @@ import com.google.gson.Gson; import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.commons.codec.binary.Base64; -import org.apache.commons.lang3.reflect.MethodUtils; +import org.apache.commons.lang.reflect.MethodUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; @@ -82,7 +82,7 @@ public static void posixFadviseIfPossible(String identifier, java.io.FileDescrip posixFadviseIfPossible.invoke(cacheManipulator, identifier, fd, offset, len, flags); } catch (Throwable t) { nativeIOPossible = false; - LOG.warn("Failed to manage OS cache for " + identifier + ", fd: " + fd.valid() + ", offset: " + offset + ", len: " + len + ", flags: " + flags + ", msg: " + t.getMessage(), t); + LOG.warn("Failed to manage OS cache for " + identifier, t); } } } @@ -111,10 +111,6 @@ private static boolean loadNativeIO() { if (posixFadviseIfPossible == null) { loaded = false; - } else { - final StringBuilder sb = new StringBuilder("posixFadviseIfPossible: "); - Arrays.stream(posixFadviseIfPossible.getParameterTypes()).forEach(p -> sb.append(p.getCanonicalName()).append(", ")); - LOG.info(sb); } return loaded; } diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java index 503f8b8ee0..8ecf60eeb7 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java @@ -165,10 +165,9 @@ public void transferSuccessful() { if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null && channelOpen) { try { PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), -// NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), NativeIO.POSIX.POSIX_FADV_DONTNEED); } catch (Throwable t) { - LOG.warn("Failed to manage OS cache for " + identifier + ", valid fd? " + fd.valid() + ", pos: " + getPosition() + ", count: " + getCount() + ", channelOpen: " + channelOpen, t); + LOG.warn("Failed to manage OS cache for " + identifier, t); } } } From 60a784ab56c9008fca33f832e5b5735107156c6b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 13:14:25 +0900 Subject: [PATCH 15/20] Remove reflection --- .../tajo/pullserver/FadvisedFileRegion.java | 1 + .../tajo/pullserver/PullServerUtil.java | 41 +++---------------- .../apache/tajo/yarn/FadvisedFileRegion.java | 3 +- 3 files changed, 8 insertions(+), 37 deletions(-) diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java index 643d9e0b29..07f0e569fb 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java @@ -151,6 +151,7 @@ long customShuffleTransfer(WritableByteChannel target, long position) protected void deallocate() { if (readaheadRequest != null) { readaheadRequest.cancel(); + readaheadRequest = null; } super.deallocate(); } diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java index 9fcea39160..75f608069a 100644 --- a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java +++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java @@ -23,7 +23,6 @@ import com.google.gson.Gson; import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.commons.codec.binary.Base64; -import org.apache.commons.lang.reflect.MethodUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; @@ -43,10 +42,12 @@ import org.apache.tajo.util.Pair; import java.io.*; -import java.lang.reflect.Method; import java.net.URI; import java.net.URLEncoder; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -55,11 +56,9 @@ public class PullServerUtil { private static final Log LOG = LogFactory.getLog(PullServerUtil.class); private static boolean nativeIOPossible = false; - private static Method posixFadviseIfPossible; - private static Object cacheManipulator; static { - if (NativeIO.isAvailable() && loadNativeIO()) { + if (NativeIO.isAvailable()) { nativeIOPossible = true; } else { LOG.warn("Unable to load hadoop nativeIO"); @@ -79,7 +78,7 @@ public static void posixFadviseIfPossible(String identifier, java.io.FileDescrip long offset, long len, int flags) { if (nativeIOPossible) { try { - posixFadviseIfPossible.invoke(cacheManipulator, identifier, fd, offset, len, flags); + NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, offset, len, flags); } catch (Throwable t) { nativeIOPossible = false; LOG.warn("Failed to manage OS cache for " + identifier, t); @@ -87,34 +86,6 @@ public static void posixFadviseIfPossible(String identifier, java.io.FileDescrip } } - /* load hadoop native method if possible */ - private static boolean loadNativeIO() { - boolean loaded = true; - if (nativeIOPossible) return loaded; - - Class[] parameters = {String.class, FileDescriptor.class, Long.TYPE, Long.TYPE, Integer.TYPE}; - try { - Method getCacheManipulator = MethodUtils.getAccessibleMethod(NativeIO.POSIX.class, "getCacheManipulator", new Class[0]); - Class posixClass; - if (getCacheManipulator != null) { - cacheManipulator = MethodUtils.invokeStaticMethod(NativeIO.POSIX.class, "getCacheManipulator", null); - posixClass = cacheManipulator.getClass(); - } else { - cacheManipulator = null; - posixClass = NativeIO.POSIX.class; - } - posixFadviseIfPossible = MethodUtils.getAccessibleMethod(posixClass, "posixFadviseIfPossible", parameters); - } catch (Throwable e) { - loaded = false; - LOG.warn("Failed to access posixFadviseIfPossible :" + e.getMessage(), e); - } - - if (posixFadviseIfPossible == null) { - loaded = false; - } - return loaded; - } - public static Path getBaseOutputDir(String queryId, String executionBlockSequenceId) { return StorageUtil.concatPath( queryId, diff --git a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java index 8ecf60eeb7..17a753fa22 100644 --- a/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java +++ b/tajo-yarn/src/main/java/org/apache/tajo/yarn/FadvisedFileRegion.java @@ -161,8 +161,7 @@ public void releaseExternalResources() { * we don't need the region to be cached anymore. */ public void transferSuccessful() { - final boolean channelOpen = fileChannel.isOpen(); - if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null && channelOpen) { + if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0 && fileChannel != null) { try { PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(), NativeIO.POSIX.POSIX_FADV_DONTNEED); From 07dde34597f6d3c55cac2b3bdac2917c95bf4701 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 15:22:46 +0900 Subject: [PATCH 16/20] Fix array out of index bounds --- .../org/apache/tajo/worker/AbstractFetcher.java | 5 +++++ .../org/apache/tajo/worker/LocalFetcher.java | 17 ++++++++++++++--- .../org/apache/tajo/worker/RemoteFetcher.java | 1 + 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java index b36c5fc8ad..a12db77fd6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractFetcher.java @@ -37,6 +37,7 @@ public abstract class AbstractFetcher { protected long startTime; protected volatile long finishTime; + protected int fileNum; protected long fileLen; protected int messageReceiveCount; @@ -67,6 +68,10 @@ public long getFileLen() { return fileLen; } + public int getFileNum() { + return fileNum; + } + public TajoProtos.FetcherState getState() { return state; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java index 4cf7307797..932a8b6a36 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java @@ -185,7 +185,13 @@ private List getWithInternalPullServer() throws IOException { } fileChunks.stream().forEach(c -> c.setEbId(tableName)); endFetch(FetcherState.FETCH_DATA_FINISHED); - fileLen = fileChunks.get(0).getFile().length(); + if (fileChunks.size() > 0) { + fileLen = fileChunks.get(0).getFile().length(); + fileNum = 1; + } else { + fileNum = 0; + fileLen = 0; + } return fileChunks; } @@ -230,6 +236,8 @@ private List getChunksForHashShuffle(final PullServerParams params, f chunk.setEbId(tableName); chunk.setFromRemote(false); fileChunks.add(chunk); + fileLen = file.length(); + fileNum = 1; } endFetch(FetcherState.FETCH_DATA_FINISHED); @@ -280,6 +288,7 @@ private List getChunksForRangeShuffle(final PullServerParams params, endFetch(FetcherState.FETCH_FAILED); } else { state = FetcherState.FETCH_DATA_FETCHING; + fileLen = fileNum = 0; for (FileChunkMeta eachMeta : chunkMetas) { Path outputPath = StorageUtil.concatPath(queryBaseDir, eachMeta.getTaskId(), "output"); if (!localDirAllocator.ifExists(outputPath.toString(), conf)) { @@ -287,10 +296,12 @@ private List getChunksForRangeShuffle(final PullServerParams params, continue; } Path path = localFileSystem.makeQualified(localDirAllocator.getLocalPathToRead(outputPath.toString(), conf)); - FileChunk chunk = new FileChunk(new File(URI.create(path.toUri() + "/output")), - eachMeta.getStartOffset(), eachMeta.getLength()); + File file = new File(URI.create(path.toUri() + "/output")); + FileChunk chunk = new FileChunk(file, eachMeta.getStartOffset(), eachMeta.getLength()); chunk.setEbId(tableName); fileChunks.add(chunk); + fileLen += file.length(); + fileNum++; } endFetch(FetcherState.FETCH_DATA_FINISHED); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java index 9328c977de..9a8a70bb58 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/RemoteFetcher.java @@ -243,6 +243,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) if (msg instanceof LastHttpContent) { if (raf != null) { fileLen = file.length(); + fileNum = 1; } if (totalReceivedContentLength == length) { From bf3fc1ecdf6cb148610a881a80cec380d6716af7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 15:38:31 +0900 Subject: [PATCH 17/20] Fix broken web UI --- tajo-core/src/main/resources/webapps/worker/taskdetail.jsp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp index 88b7c2430e..3eab5cdbbd 100644 --- a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp +++ b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp @@ -25,7 +25,7 @@ <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TajoIdUtils" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> -<%@ page import="org.apache.tajo.worker.Fetcher" %> +<%@ page import="org.apache.tajo.worker.AbstractFetcher" %> <%@ page import="org.apache.tajo.worker.TajoWorker" %> <%@ page import="org.apache.tajo.worker.Task" %> <%@ page import="org.apache.tajo.worker.TaskHistory" %> @@ -161,7 +161,7 @@ URI <% - for (Fetcher eachFetcher : task.getFetchers()) { + for (AbstractFetcher eachFetcher : task.getFetchers()) { %> <%=index%> From e259146da5c5230c51826cbbddf9a830e88eadcd Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 15:46:40 +0900 Subject: [PATCH 18/20] Add missing start time --- .../src/main/java/org/apache/tajo/worker/LocalFetcher.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java index 932a8b6a36..1cedbc55ef 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java @@ -170,12 +170,12 @@ public LocalFetcher(TajoConf conf, URI uri, ExecutionBlockContext executionBlock @Override public List get() throws IOException { + this.startTime = System.currentTimeMillis(); return pullServerService != null ? getWithInternalPullServer() : getWithExternalPullServer(); } private List getWithInternalPullServer() throws IOException { final List fileChunks = new ArrayList<>(); - startTime = System.currentTimeMillis(); PullServerParams params = new PullServerParams(uri.toString()); try { fileChunks.addAll(pullServerService.getFileChunks(conf, localDirAllocator, params)); @@ -253,7 +253,6 @@ private List getChunksForRangeShuffle(final PullServerParams params, bootstrap.handler(initializer); } - this.startTime = System.currentTimeMillis(); this.state = FetcherState.FETCH_META_FETCHING; ChannelFuture future = null; try { From a9fb2af3c0f8a00b887ee90afdef4c2cdd5ca0b8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 15:57:04 +0900 Subject: [PATCH 19/20] Fix wrong chunk length --- .../src/main/java/org/apache/tajo/worker/LocalFetcher.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java index 1cedbc55ef..e346bdff6f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LocalFetcher.java @@ -186,7 +186,7 @@ private List getWithInternalPullServer() throws IOException { fileChunks.stream().forEach(c -> c.setEbId(tableName)); endFetch(FetcherState.FETCH_DATA_FINISHED); if (fileChunks.size() > 0) { - fileLen = fileChunks.get(0).getFile().length(); + fileLen = fileChunks.get(0).length(); fileNum = 1; } else { fileNum = 0; @@ -236,7 +236,7 @@ private List getChunksForHashShuffle(final PullServerParams params, f chunk.setEbId(tableName); chunk.setFromRemote(false); fileChunks.add(chunk); - fileLen = file.length(); + fileLen = chunk.length(); fileNum = 1; } @@ -299,7 +299,7 @@ private List getChunksForRangeShuffle(final PullServerParams params, FileChunk chunk = new FileChunk(file, eachMeta.getStartOffset(), eachMeta.getLength()); chunk.setEbId(tableName); fileChunks.add(chunk); - fileLen += file.length(); + fileLen += chunk.length(); fileNum++; } endFetch(FetcherState.FETCH_DATA_FINISHED); From e622ea630b2af868a3e3a7addeb651aa0a941d4d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 26 Apr 2016 18:36:48 +0900 Subject: [PATCH 20/20] Documentation --- tajo-docs/src/main/sphinx/configuration.rst | 1 + .../sphinx/configuration/cluster_setup.rst | 18 ++--- .../pullserver_configuration.rst | 75 +++++++++++++++++++ 3 files changed, 85 insertions(+), 9 deletions(-) create mode 100644 tajo-docs/src/main/sphinx/configuration/pullserver_configuration.rst diff --git a/tajo-docs/src/main/sphinx/configuration.rst b/tajo-docs/src/main/sphinx/configuration.rst index 8b0b8a2e66..bef49cb7d2 100644 --- a/tajo-docs/src/main/sphinx/configuration.rst +++ b/tajo-docs/src/main/sphinx/configuration.rst @@ -10,6 +10,7 @@ Configuration configuration/tajo_master_configuration configuration/worker_configuration configuration/catalog_configuration + configuration/pullserver_configuration configuration/ha_configuration configuration/service_config_defaults configuration/tajo-site-xml diff --git a/tajo-docs/src/main/sphinx/configuration/cluster_setup.rst b/tajo-docs/src/main/sphinx/configuration/cluster_setup.rst index e9d17eb25a..3835512ec6 100644 --- a/tajo-docs/src/main/sphinx/configuration/cluster_setup.rst +++ b/tajo-docs/src/main/sphinx/configuration/cluster_setup.rst @@ -1,9 +1,9 @@ -******************************************* +************* Cluster Setup -******************************************* +************* Fully Distributed Mode -========================================== +====================== A fully distributed mode enables a Tajo instance to run on `Hadoop Distributed File System (HDFS) `_. In this mode, a number of Tajo workers run across a number of the physical nodes where HDFS data nodes run. @@ -11,7 +11,7 @@ In this section, we explain how to setup the cluster mode. Settings --------------------------------------------------------- +-------- Please add the following configs to tajo-site.xml file: @@ -43,7 +43,7 @@ Please add the following configs to tajo-site.xml file: Workers --------------------------------------------------------- +------- The file ``conf/workers`` lists all host names of workers, one per line. By default, this file contains the single entry ``localhost``. @@ -59,7 +59,7 @@ For example: :: Make base directories and set permissions --------------------------------------------------------- +----------------------------------------- If you want to know Tajo’s configuration in more detail, see Configuration page. Before launching the tajo, you should create the tajo root dir and set the permission as follows: :: @@ -69,7 +69,7 @@ Before launching the tajo, you should create the tajo root dir and set the permi Launch a Tajo cluster --------------------------------------------------------- +--------------------- Then, execute ``start-tajo.sh`` :: @@ -77,10 +77,10 @@ Then, execute ``start-tajo.sh`` :: .. note:: - In default, each worker is set to very little resource capacity. In order to increase parallel degree, please read + By default, each worker is set to very little resource capacity. In order to increase parallel degree, please read :doc:`/configuration/worker_configuration`. .. note:: - In default, TajoMaster listens on 127.0.0.1 for clients. To allow remote clients to access TajoMaster, please set tajo.master.client-rpc.address config to tajo-site.xml. In order to know how to change the listen port, please refer :doc:`/configuration/service_config_defaults`. + By default, TajoMaster listens on 127.0.0.1 for clients. To allow remote clients to access TajoMaster, please set tajo.master.client-rpc.address config to tajo-site.xml. In order to know how to change the listen port, please refer :doc:`/configuration/service_config_defaults`. diff --git a/tajo-docs/src/main/sphinx/configuration/pullserver_configuration.rst b/tajo-docs/src/main/sphinx/configuration/pullserver_configuration.rst new file mode 100644 index 0000000000..68760fca80 --- /dev/null +++ b/tajo-docs/src/main/sphinx/configuration/pullserver_configuration.rst @@ -0,0 +1,75 @@ +************************* +Pull Server Configuration +************************* + +Pull servers are responsible for transmitting data among Tajo workers during shuffle phases. Tajo provides several modes +for pull servers. + +Internal Mode (Default) +======================= + +With the internal mode, each worker acts as a pull server. So, they need to transmit data during shuffle phase as well +as processing them during processing phase. + +Standalone Mode +=============== + +Sometimes, data shuffling requires huge memory space and a lot of cpu processing. +This can make query processing slow because Tajo's query engine should contend for limited resources with pull servers. +Tajo provides the standalone mode to avoid this unnecessary contention. + +In this mode, each pull server is executed as a separate process. To enable this mode, you need to add the following +line to ``${TAJO_HOME}/conf/tajo-env.sh``. + +.. code-block:: sh + + export TAJO_PULLSERVER_STANDALONE=true + +Then, you can see the following messages when you start up the tajo cluster. + +.. code-block:: sh + + Starting single TajoMaster + starting master, logging to ... + 192.168.10.1: starting pullserver, logging to ... + 192.168.10.1: starting worker, logging to ... + 192.168.10.2: starting pullserver, logging to ... + 192.168.10.2: starting worker, logging to ... + ... + +.. warning:: + + Currently, only one single server should be run in each machine. + +Yarn Auxiliary Service Mode +=========================== + +You can run pull servers as one of Yarn's auxiliary services. To do so, you need to add the following configurations +to ``${HADOOP_CONF}/yarn-site.xml``. + +.. code-block:: xml + + + yarn.nodemanager.aux-services + mapreduce_shuffle,tajo_shuffle + + + + yarn.nodemanager.aux-services.tajo_shuffle.class + org.apache.tajo.yarn.TajoPullServerService + + + + tajo.pullserver.port + port number + + +Optionally, you can add the below configuration to specify temp directories. For this configuration, +please refer to :doc:`/configuration/worker_configuration`. + +.. code-block:: xml + + + tajo.worker.tmpdir.locations + /path/to/tajo/temporal/directory + \ No newline at end of file