Repository: asterixdb Updated Branches: refs/heads/master f86a25b86 -> 604c921ae
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java deleted file mode 100644 index 6f3b667..0000000 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java +++ /dev/null @@ -1,146 +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.asterix.external.feed.management; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.logging.Level; -import java.util.logging.Logger; - -import org.apache.asterix.active.ActiveEvent; -import org.apache.asterix.active.ActiveLifecycleListener; -import org.apache.asterix.active.ActivityState; -import org.apache.asterix.active.EntityId; -import org.apache.asterix.active.IActiveEventSubscriber; -import org.apache.asterix.active.message.ActivePartitionMessage; -import org.apache.asterix.common.dataflow.ICcApplicationContext; -import org.apache.asterix.common.metadata.IDataset; -import org.apache.asterix.external.feed.watch.FeedEventSubscriber; -import org.apache.asterix.external.feed.watch.NoOpSubscriber; -import org.apache.hyracks.api.client.IHyracksClientConnection; -import org.apache.hyracks.api.exceptions.HyracksDataException; -import org.apache.hyracks.api.job.JobStatus; - -public class FeedEventsListener extends ActiveEntityEventsListener { - // constants - private static final Logger LOGGER = Logger.getLogger(FeedEventsListener.class.getName()); - // members - private final ICcApplicationContext appCtx; - private final String[] sources; - private final List<IActiveEventSubscriber> subscribers; - private int numRegistered; - - public FeedEventsListener(ICcApplicationContext appCtx, EntityId entityId, List<IDataset> datasets, - String[] sources) { - this.appCtx = appCtx; - this.entityId = entityId; - this.datasets = datasets; - this.sources = sources; - subscribers = new ArrayList<>(); - state = ActivityState.STOPPED; - } - - @Override - public synchronized void notify(ActiveEvent event) { - try { - switch (event.getEventKind()) { - case JOB_STARTED: - start(event); - break; - case JOB_FINISHED: - finish(); - break; - case PARTITION_EVENT: - partition((ActivePartitionMessage) event.getEventObject()); - break; - default: - LOGGER.log(Level.WARNING, "Unhandled feed event notification: " + event); - break; - } - notifySubscribers(event); - } catch (Exception e) { - LOGGER.log(Level.SEVERE, "Unhandled Exception", e); - } - } - - private synchronized void notifySubscribers(ActiveEvent event) { - notifyAll(); - Iterator<IActiveEventSubscriber> it = subscribers.iterator(); - while (it.hasNext()) { - IActiveEventSubscriber subscriber = it.next(); - if (subscriber.done()) { - it.remove(); - } else { - subscriber.notify(event); - if (subscriber.done()) { - it.remove(); - } - } - } - } - - private void partition(ActivePartitionMessage message) { - if (message.getEvent() == ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED) { - numRegistered++; - if (numRegistered == getSources().length) { - state = ActivityState.STARTED; - } - } - } - - private void finish() throws Exception { - IHyracksClientConnection hcc = appCtx.getHcc(); - JobStatus status = hcc.getJobStatus(jobId); - state = status.equals(JobStatus.FAILURE) ? ActivityState.FAILED : ActivityState.STOPPED; - ActiveLifecycleListener activeLcListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener(); - activeLcListener.getNotificationHandler().removeListener(this); - } - - private void start(ActiveEvent event) { - this.jobId = event.getJobId(); - state = ActivityState.STARTING; - } - - @Override - public IActiveEventSubscriber subscribe(ActivityState state) throws HyracksDataException { - if (state != ActivityState.STARTED && state != ActivityState.STOPPED) { - throw new HyracksDataException("Can only wait for STARTED or STOPPED state"); - } - synchronized (this) { - if (this.state == ActivityState.FAILED) { - throw new HyracksDataException("Feed has failed"); - } else if (this.state == state) { - return NoOpSubscriber.INSTANCE; - } - return doSubscribe(state); - } - } - - // Called within synchronized block - private FeedEventSubscriber doSubscribe(ActivityState state) { - FeedEventSubscriber subscriber = new FeedEventSubscriber(this, state); - subscribers.add(subscriber); - return subscriber; - } - - public String[] getSources() { - return sources; - } -} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java index c71b8a2..e6ac265 100644 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java @@ -34,6 +34,7 @@ public class AdapterExecutor implements Runnable { // increase or decrease at any time) private final FeedAdapter adapter; // The adapter private final AdapterRuntimeManager adapterManager;// The runtime manager <-- two way visibility --> + private int restartCount = 0; public AdapterExecutor(IFrameWriter writer, FeedAdapter adapter, AdapterRuntimeManager adapterManager) { this.writer = writer; @@ -81,8 +82,13 @@ public class AdapterExecutor implements Runnable { LOGGER.error("Exception during feed ingestion ", e); continueIngestion = adapter.handleException(e); failedIngestion = !continueIngestion; + restartCount++; } } return failedIngestion; } + + public String getStats() { + return "{\"adapter-stats\": " + adapter.getStats() + ", \"executor-restart-times\": " + restartCount + "}"; + } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java index 6214d9f..1b5eeac 100644 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java @@ -137,4 +137,8 @@ public class AdapterRuntimeManager { public void setDone(boolean done) { this.done = done; } + + public String getStats() { + return adapterExecutor.getStats(); + } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java deleted file mode 100644 index 590af01..0000000 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java +++ /dev/null @@ -1,62 +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.asterix.external.feed.runtime; - -import java.util.logging.Level; -import java.util.logging.Logger; - -import org.apache.asterix.active.ActiveRuntimeId; -import org.apache.asterix.active.EntityId; -import org.apache.asterix.active.IActiveRuntime; -import org.apache.hyracks.api.exceptions.HyracksDataException; - -public class IngestionRuntime implements IActiveRuntime { - - private static final Logger LOGGER = Logger.getLogger(IngestionRuntime.class.getName()); - - private final AdapterRuntimeManager adapterRuntimeManager; - private final ActiveRuntimeId runtimeId; - private final EntityId feedId; - - public IngestionRuntime(EntityId entityId, ActiveRuntimeId runtimeId, AdapterRuntimeManager adaptorRuntimeManager) { - this.feedId = entityId; - this.runtimeId = runtimeId; - this.adapterRuntimeManager = adaptorRuntimeManager; - } - - @Override - public ActiveRuntimeId getRuntimeId() { - return this.runtimeId; - } - - public void start() { - adapterRuntimeManager.start(); - LOGGER.log(Level.INFO, "Feed " + feedId.getEntityName() + " running on partition " + runtimeId); - } - - @Override - public void stop() throws InterruptedException, HyracksDataException { - adapterRuntimeManager.stop(); - LOGGER.log(Level.INFO, "Feed " + feedId.getEntityName() + " stopped on partition " + runtimeId); - } - - public EntityId getFeedId() { - return feedId; - } -} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java new file mode 100644 index 0000000..822d725 --- /dev/null +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java @@ -0,0 +1,56 @@ +/* + * 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.asterix.external.feed.watch; + +import org.apache.asterix.active.IActiveEntityEventsListener; +import org.apache.asterix.active.IActiveEventSubscriber; +import org.apache.hyracks.api.exceptions.HyracksDataException; + +public abstract class AbstractSubscriber implements IActiveEventSubscriber { + + protected final IActiveEntityEventsListener listener; + private boolean done = false; + + public AbstractSubscriber(IActiveEntityEventsListener listener) { + this.listener = listener; + } + + @Override + public synchronized boolean isDone() { + return done; + } + + public synchronized void complete() throws HyracksDataException { + done = true; + notifyAll(); + } + + @Override + public synchronized void sync() throws InterruptedException { + while (!done) { + wait(); + } + } + + @Override + public synchronized void unsubscribe() { + done = true; + notifyAll(); + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java deleted file mode 100644 index 0e931f7..0000000 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java +++ /dev/null @@ -1,64 +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.asterix.external.feed.watch; - -import org.apache.asterix.active.ActiveEvent; -import org.apache.asterix.active.ActivityState; -import org.apache.asterix.active.IActiveEventSubscriber; -import org.apache.asterix.external.feed.management.FeedEventsListener; - -public class FeedEventSubscriber implements IActiveEventSubscriber { - - private final FeedEventsListener listener; - private final ActivityState state; - private boolean done = false; - - public FeedEventSubscriber(FeedEventsListener listener, ActivityState state) { - this.listener = listener; - this.state = state; - - } - - @Override - public synchronized void notify(ActiveEvent event) { - if (listener.getState() == state || listener.getState() == ActivityState.FAILED - || listener.getState() == ActivityState.STOPPED) { - done = true; - notifyAll(); - } - } - - @Override - public synchronized boolean done() { - return done; - } - - @Override - public synchronized void sync() throws InterruptedException { - while (!done) { - wait(); - } - } - - @Override - public synchronized void unsubscribe() { - done = true; - notifyAll(); - } -} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java index 9d8c570..42f7a74 100644 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java @@ -19,7 +19,9 @@ package org.apache.asterix.external.feed.watch; import org.apache.asterix.active.ActiveEvent; +import org.apache.asterix.active.IActiveEntityEventsListener; import org.apache.asterix.active.IActiveEventSubscriber; +import org.apache.hyracks.api.exceptions.HyracksDataException; /** * An event subscriber that does not listen to any events @@ -33,22 +35,26 @@ public class NoOpSubscriber implements IActiveEventSubscriber { @Override public void notify(ActiveEvent event) { - // do nothing + // no op } @Override - public boolean done() { + public boolean isDone() { return true; } @Override public void sync() { - // do nothing + // no op } @Override public void unsubscribe() { - // do nothing + // no op } + @Override + public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException { + // no op + } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java new file mode 100644 index 0000000..fa2fa7f --- /dev/null +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java @@ -0,0 +1,43 @@ +/* + * 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.asterix.external.feed.watch; + +import org.apache.asterix.active.ActiveEvent; +import org.apache.asterix.active.IActiveEntityEventsListener; +import org.apache.hyracks.api.exceptions.HyracksDataException; + +public class StatsSubscriber extends AbstractSubscriber { + + public StatsSubscriber(IActiveEntityEventsListener listener) throws HyracksDataException { + super(listener); + listener.subscribe(this); + } + + @Override + public void notify(ActiveEvent event) throws HyracksDataException { + if (event.getEventKind() == ActiveEvent.Kind.STATS_UPDATED) { + complete(); + } + } + + @Override + public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException { + //Does nothing upon subscription + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java new file mode 100644 index 0000000..ea7e3ae --- /dev/null +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java @@ -0,0 +1,58 @@ +/* + * 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.asterix.external.feed.watch; + +import org.apache.asterix.active.ActiveEvent; +import org.apache.asterix.active.ActivityState; +import org.apache.asterix.active.IActiveEntityEventsListener; +import org.apache.asterix.common.exceptions.ErrorCode; +import org.apache.asterix.common.exceptions.RuntimeDataException; +import org.apache.hyracks.api.exceptions.HyracksDataException; + +public class WaitForStateSubscriber extends AbstractSubscriber { + + private final ActivityState targetState; + + public WaitForStateSubscriber(IActiveEntityEventsListener listener, ActivityState targetState) + throws HyracksDataException { + super(listener); + this.targetState = targetState; + if (targetState != ActivityState.STARTED && targetState != ActivityState.STOPPED) { + throw new RuntimeDataException(ErrorCode.CANNOT_WAIT_FOR_STATE, targetState); + } + listener.subscribe(this); + } + + @Override + public void notify(ActiveEvent event) throws HyracksDataException { + if (listener.getState() == targetState) { + complete(); + } + } + + @Override + public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException { + if (eventsListener.getState() == ActivityState.FAILED) { + throw new RuntimeDataException(ErrorCode.CANNOT_SUBSCRIBE_TO_FAILED_ACTIVE_ENTITY); + } + if (listener.getState() == targetState) { + complete(); + } + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java index 2876ea6..8c6a420 100644 --- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java +++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java @@ -105,4 +105,13 @@ public class FeedIntakeOperatorNodePushable extends ActiveSourceOperatorNodePush adapterRuntimeManager.stop(); } } + + @Override + public String getStats() { + if (adapterRuntimeManager != null) { + return adapterRuntimeManager.getStats(); + } else { + return "\"Runtime stats is not available.\""; + } + } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java index 3c41165..4a4c25a 100644 --- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java +++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java @@ -91,8 +91,7 @@ public abstract class AbstractServlet implements IServlet { } } - protected void sendError(IServletResponse response, HttpResponseStatus status, String message) - throws IOException { + protected void sendError(IServletResponse response, HttpResponseStatus status, String message) throws IOException { response.setStatus(status); HttpUtil.setContentType(response, HttpUtil.ContentType.TEXT_PLAIN, HttpUtil.Encoding.UTF8); if (message != null) { @@ -154,7 +153,7 @@ public abstract class AbstractServlet implements IServlet { public String localPath(IServletRequest request) { final String uri = request.getHttpRequest().uri(); - int queryStart = uri.indexOf("?"); + int queryStart = uri.indexOf('?'); return queryStart == -1 ? uri.substring(trim(uri)) : uri.substring(trim(uri), queryStart); }