Add Asterix Extension Manager
More extension support is added. A user can now provide implementations
for the IExtension interface which will give them more control over
the behavior of the system and give them the ability to add custom
features.
Initial customizations include:
1. Metadata Tuple Translators
2. Metadata Datasets
3. Query Translators
4. Statement Handlers
5. Lang Compilation Provider
Change-Id: I280268495cc3aad00f898cba21f7299f7120ce5c
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1017
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <tillw@apache.org>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
index e5ccd05..c907a36 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
@@ -18,16 +18,14 @@
*/
package org.apache.asterix.active;
-import java.io.Serializable;
-
import org.apache.hyracks.api.job.JobId;
public class ActiveEvent {
private final JobId jobId;
private final EntityId entityId;
- private final Serializable payload;
private final EventKind eventKind;
+ private final Object eventObject;
public enum EventKind {
JOB_START,
@@ -35,34 +33,30 @@
PARTITION_EVENT
}
- public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind) {
- this(jobId, eventKind, null, null);
- }
-
- public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind, EntityId feedId) {
- this(jobId, eventKind, feedId, null);
- }
-
- public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind, EntityId feedId, Serializable payload) {
+ public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind, EntityId entityId, Object eventObject) {
this.jobId = jobId;
+ this.entityId = entityId;
this.eventKind = eventKind;
- this.entityId = feedId;
- this.payload = payload;
+ this.eventObject = eventObject;
+ }
+
+ public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind, EntityId entityId) {
+ this(jobId, eventKind, entityId, null);
}
public JobId getJobId() {
return jobId;
}
- public EntityId getFeedId() {
+ public EntityId getEntityId() {
return entityId;
}
- public Serializable getPayload() {
- return payload;
- }
-
public EventKind getEventKind() {
return eventKind;
}
+
+ public Object getEventObject() {
+ return eventObject;
+ }
}
\ No newline at end of file
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java
index 9c69aca..30a2eb6 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java
@@ -55,8 +55,8 @@
EntityId entityId = jobInfo.getEntityId();
IActiveEntityEventsListener listener = entityEventListener.get(entityId);
if (DEBUG) {
- LOGGER.log(Level.INFO, "Next event is of type " + event.getEventKind());
- LOGGER.log(Level.INFO, "Notifying the listener");
+ LOGGER.log(Level.WARNING, "Next event is of type " + event.getEventKind());
+ LOGGER.log(Level.WARNING, "Notifying the listener");
}
listener.notify(event);
if (event.getEventKind() == EventKind.JOB_FINISH) {
@@ -74,7 +74,7 @@
private void removeFinishedJob(JobId jobId) {
if (DEBUG) {
- LOGGER.log(Level.INFO, "Removing the job");
+ LOGGER.log(Level.WARNING, "Removing the job");
}
jobId2ActiveJobInfos.remove(jobId);
}
@@ -82,7 +82,7 @@
private void removeInactiveListener(IActiveEntityEventsListener listener) {
if (!listener.isEntityActive()) {
if (DEBUG) {
- LOGGER.log(Level.INFO, "Removing the listener since it is not active anymore");
+ LOGGER.log(Level.WARNING, "Removing the listener since it is not active anymore");
}
entityEventListener.remove(listener.getEntityId());
}
@@ -90,40 +90,45 @@
public IActiveEntityEventsListener getActiveEntityListener(EntityId entityId) {
if (DEBUG) {
- LOGGER.log(Level.INFO, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
+ LOGGER.log(Level.WARNING, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
IActiveEntityEventsListener listener = entityEventListener.get(entityId);
- LOGGER.log(Level.INFO, "Listener found: " + listener);
+ LOGGER.log(Level.WARNING, "Listener found: " + listener);
}
return entityEventListener.get(entityId);
}
public synchronized ActiveJob[] getActiveJobs() {
if (DEBUG) {
- LOGGER.log(Level.INFO, "getActiveJobs() was called");
- LOGGER.log(Level.INFO, "Number of jobs found: " + jobId2ActiveJobInfos.size());
+ LOGGER.log(Level.WARNING, "getActiveJobs() was called");
+ LOGGER.log(Level.WARNING, "Number of jobs found: " + jobId2ActiveJobInfos.size());
}
return jobId2ActiveJobInfos.values().toArray(new ActiveJob[jobId2ActiveJobInfos.size()]);
}
public boolean isActiveJob(JobId jobId) {
if (DEBUG) {
- LOGGER.log(Level.INFO, "isActiveJob(JobId jobId) called with jobId: " + jobId);
+ LOGGER.log(Level.WARNING, "isActiveJob(JobId jobId) called with jobId: " + jobId);
boolean found = jobId2ActiveJobInfos.get(jobId) != null;
- LOGGER.log(Level.INFO, "Job was found to be: " + (found ? "Active" : "Inactive"));
+ LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
}
return jobId2ActiveJobInfos.get(jobId) != null;
}
+ public EntityId getEntity(JobId jobId) {
+ ActiveJob jobInfo = jobId2ActiveJobInfos.get(jobId);
+ return jobInfo == null ? null : jobInfo.getEntityId();
+ }
+
public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) {
if (DEBUG) {
- LOGGER.log(Level.INFO,
+ LOGGER.log(Level.WARNING,
"notifyJobCreation(JobId jobId, JobSpecification jobSpecification) was called with jobId = "
+ jobId);
}
Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
if (property == null || !(property instanceof ActiveJob)) {
if (DEBUG) {
- LOGGER.log(Level.INFO, "Job was is not active. property found to be: " + property);
+ LOGGER.log(Level.WARNING, "Job was is not active. property found to be: " + property);
}
return;
} else {
@@ -131,7 +136,7 @@
}
if (DEBUG) {
boolean found = jobId2ActiveJobInfos.get(jobId) != null;
- LOGGER.log(Level.INFO, "Job was found to be: " + (found ? "Active" : "Inactive"));
+ LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
}
ActiveJob jobInfo = jobId2ActiveJobInfos.get(jobId);
if (jobInfo != null) {
@@ -139,11 +144,12 @@
IActiveEntityEventsListener listener = entityEventListener.get(entityId);
listener.notifyJobCreation(jobId, jobSpecification);
if (DEBUG) {
- LOGGER.log(Level.INFO, "Listener was notified" + jobId);
+ LOGGER.log(Level.WARNING, "Listener was notified" + jobId);
}
} else {
if (DEBUG) {
- LOGGER.log(Level.INFO, "Listener was not notified since it was not registered for the job " + jobId);
+ LOGGER.log(Level.WARNING,
+ "Listener was not notified since it was not registered for the job " + jobId);
}
}
}
@@ -154,16 +160,17 @@
public synchronized IActiveEntityEventsListener[] getEventListeners() {
if (DEBUG) {
- LOGGER.log(Level.INFO, "getEventListeners() was called");
- LOGGER.log(Level.INFO, "returning " + entityEventListener.size() + " Listeners");
+ LOGGER.log(Level.WARNING, "getEventListeners() was called");
+ LOGGER.log(Level.WARNING, "returning " + entityEventListener.size() + " Listeners");
}
return entityEventListener.values().toArray(new IActiveEntityEventsListener[entityEventListener.size()]);
}
public synchronized void registerListener(IActiveEntityEventsListener listener) throws HyracksDataException {
if (DEBUG) {
- LOGGER.log(Level.INFO, "registerListener(IActiveEntityEventsListener listener) was called for the entity "
- + listener.getEntityId());
+ LOGGER.log(Level.WARNING,
+ "registerListener(IActiveEntityEventsListener listener) was called for the entity "
+ + listener.getEntityId());
}
if (entityEventListener.containsKey(listener.getEntityId())) {
throw new HyracksDataException(
@@ -174,9 +181,9 @@
public synchronized void monitorJob(JobId jobId, ActiveJob activeJob) {
if (DEBUG) {
- LOGGER.log(Level.INFO, "monitorJob(JobId jobId, ActiveJob activeJob) called with job id: " + jobId);
+ LOGGER.log(Level.WARNING, "monitorJob(JobId jobId, ActiveJob activeJob) called with job id: " + jobId);
boolean found = jobId2ActiveJobInfos.get(jobId) != null;
- LOGGER.log(Level.INFO, "Job was found to be: " + (found ? "Active" : "Inactive"));
+ LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
}
if (entityEventListener.containsKey(activeJob.getEntityId())) {
if (jobId2ActiveJobInfos.containsKey(jobId)) {
@@ -184,7 +191,7 @@
return;
}
if (DEBUG) {
- LOGGER.log(Level.INFO, "monitoring started for job id: " + jobId);
+ LOGGER.log(Level.WARNING, "monitoring started for job id: " + jobId);
}
jobId2ActiveJobInfos.put(jobId, activeJob);
} else {
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index 5992294..2f34465 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -21,29 +21,23 @@
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import org.apache.asterix.active.message.ActiveManagerMessage;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.log4j.Logger;
public class ActiveManager {
+ private static final Logger LOGGER = Logger.getLogger(ActiveManager.class.getName());
private final Map<ActiveRuntimeId, IActiveRuntime> runtimes;
-
- private final IActiveRuntimeRegistry activeRuntimeRegistry;
-
private final ConcurrentFramePool activeFramePool;
-
private final String nodeId;
public ActiveManager(String nodeId, long activeMemoryBudget, int frameSize) throws HyracksDataException {
this.nodeId = nodeId;
- this.activeRuntimeRegistry = new ActiveRuntimeRegistry(nodeId);
this.activeFramePool = new ConcurrentFramePool(nodeId, activeMemoryBudget, frameSize);
this.runtimes = new ConcurrentHashMap<>();
}
- public IActiveRuntimeRegistry getActiveRuntimeRegistry() {
- return activeRuntimeRegistry;
- }
-
public ConcurrentFramePool getFramePool() {
return activeFramePool;
}
@@ -59,12 +53,37 @@
runtimes.remove(id);
}
- public IActiveRuntime getSubscribableRuntime(ActiveRuntimeId subscribableRuntimeId) {
- return runtimes.get(subscribableRuntimeId);
+ public IActiveRuntime getRuntime(ActiveRuntimeId runtimeId) {
+ return runtimes.get(runtimeId);
}
@Override
public String toString() {
return ActiveManager.class.getSimpleName() + "[" + nodeId + "]";
}
+
+ public void submit(ActiveManagerMessage message) {
+ switch (message.getKind()) {
+ case ActiveManagerMessage.STOP_ACTIVITY:
+ stopRuntime(message);
+ break;
+ default:
+ LOGGER.warn("Unknown message type received");
+ }
+ }
+
+ private void stopRuntime(ActiveManagerMessage message) {
+ ActiveRuntimeId runtimeId = (ActiveRuntimeId) message.getPayload();
+ IActiveRuntime runtime = runtimes.get(runtimeId);
+ if (runtime == null) {
+ LOGGER.warn("Request to stop a runtime that is not registered " + runtimeId);
+ } else {
+ try {
+ runtime.stop();
+ } catch (HyracksDataException | InterruptedException e) {
+ // TODO(till) Figure out a better way to handle failure to stop a runtime
+ LOGGER.warn("Failed to stop runtime: " + runtimeId, e);
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeId.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeId.java
index 64926fd..f1f5876 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeId.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeId.java
@@ -25,20 +25,20 @@
private static final long serialVersionUID = 1L;
private final EntityId entityId;
- private final String runtimeId;
+ private final String runtimeName;
private final int partition;
private final int hashCode;
- public ActiveRuntimeId(EntityId entityId, String runtimeId, int partition) {
+ public ActiveRuntimeId(EntityId entityId, String runtimeName, int partition) {
this.entityId = entityId;
- this.runtimeId = runtimeId;
+ this.runtimeName = runtimeName;
this.partition = partition;
this.hashCode = toString().hashCode();
}
@Override
public String toString() {
- return "(" + entityId + ")" + "[" + partition + "]:" + runtimeId;
+ return "(" + entityId + ")" + "[" + partition + "]:" + runtimeName;
}
@Override
@@ -50,7 +50,7 @@
return false;
}
ActiveRuntimeId other = (ActiveRuntimeId) o;
- return other.entityId.equals(entityId) && other.getFeedRuntimeType().equals(runtimeId)
+ return other.entityId.equals(entityId) && other.getRuntimeName().equals(runtimeName)
&& other.getPartition() == partition;
}
@@ -59,19 +59,15 @@
return hashCode;
}
- public String getFeedRuntimeType() {
- return runtimeId;
+ public String getRuntimeName() {
+ return runtimeName;
}
public int getPartition() {
return partition;
}
- public String getRuntimeType() {
- return runtimeId;
- }
-
- public EntityId getFeedId() {
+ public EntityId getEntityId() {
return entityId;
}
}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java
index 9743856..e71367a 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java
@@ -31,7 +31,7 @@
public class ActiveRuntimeManager {
private static final Logger LOGGER = Logger.getLogger(ActiveRuntimeManager.class.getName());
- private final Map<ActiveRuntimeId, ActiveRuntime> activeRuntimes;
+ private final Map<ActiveRuntimeId, ActiveSourceOperatorNodePushable> activeRuntimes;
private final ExecutorService executorService;
@@ -61,11 +61,11 @@
}
}
- public ActiveRuntime getFeedRuntime(ActiveRuntimeId runtimeId) {
+ public ActiveSourceOperatorNodePushable getRuntime(ActiveRuntimeId runtimeId) {
return activeRuntimes.get(runtimeId);
}
- public void registerRuntime(ActiveRuntimeId runtimeId, ActiveRuntime feedRuntime) {
+ public void registerRuntime(ActiveRuntimeId runtimeId, ActiveSourceOperatorNodePushable feedRuntime) {
activeRuntimes.put(runtimeId, feedRuntime);
}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeRegistry.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeRegistry.java
deleted file mode 100644
index 050426c..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeRegistry.java
+++ /dev/null
@@ -1,79 +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.active;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * An implementation of the {@code IActiveRuntimeRegistry} interface.
- * Provider necessary central repository for registering/retrieving
- * artifacts/services associated with an active entity.
- */
-public class ActiveRuntimeRegistry implements IActiveRuntimeRegistry {
-
- private static final Logger LOGGER = Logger.getLogger(ActiveRuntimeRegistry.class.getName());
-
- private Map<ActiveRuntimeId, ActiveRuntimeManager> activeRuntimeManagers = new HashMap<>();
- private final String nodeId;
-
- public ActiveRuntimeRegistry(String nodeId) {
- this.nodeId = nodeId;
- }
-
- @Override
- public void deregisterRuntime(ActiveRuntimeId runtimeId) {
- try {
- ActiveRuntimeManager mgr = activeRuntimeManagers.get(runtimeId);
- if (mgr != null) {
- mgr.deregisterRuntime(runtimeId);
- mgr.close();
- activeRuntimeManagers.remove(runtimeId);
- }
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.log(Level.WARNING, "Exception in closing feed runtime" + e.getMessage(), e);
- }
- }
-
- }
-
- @Override
- public synchronized void registerRuntime(ActiveRuntime runtime) {
- ActiveRuntimeManager runtimeMgr = activeRuntimeManagers.get(runtime.getRuntimeId());
- if (runtimeMgr == null) {
- runtimeMgr = new ActiveRuntimeManager();
- activeRuntimeManagers.put(runtime.getRuntimeId(), runtimeMgr);
- }
- runtimeMgr.registerRuntime(runtime.getRuntimeId(), runtime);
- }
-
- @Override
- public ActiveRuntime getRuntime(ActiveRuntimeId runtimeId) {
- ActiveRuntimeManager runtimeMgr = activeRuntimeManagers.get(runtimeId);
- return runtimeMgr != null ? runtimeMgr.getFeedRuntime(runtimeId) : null;
- }
-
- @Override
- public String toString() {
- return ActiveRuntimeRegistry.class.getSimpleName() + "[" + nodeId + "]";
- }
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
new file mode 100644
index 0000000..1cda298
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
@@ -0,0 +1,125 @@
+/*
+ * 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.active;
+
+import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
+
+public abstract class ActiveSourceOperatorNodePushable extends AbstractOperatorNodePushable implements IActiveRuntime {
+
+ protected final IHyracksTaskContext ctx;
+ protected final ActiveManager activeManager;
+ /** A unique identifier for the runtime **/
+ protected final ActiveRuntimeId runtimeId;
+ private volatile boolean done = false;
+
+ public ActiveSourceOperatorNodePushable(IHyracksTaskContext ctx, ActiveRuntimeId runtimeId) {
+ this.ctx = ctx;
+ activeManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ .getApplicationObject()).getActiveManager();
+ this.runtimeId = runtimeId;
+ }
+
+ @Override
+ public ActiveRuntimeId getRuntimeId() {
+ return runtimeId;
+ }
+
+ /**
+ * Starts the active job. This method must not return until the job has finished
+ *
+ * @throws HyracksDataException
+ * @throws InterruptedException
+ */
+ protected abstract void start() throws HyracksDataException, InterruptedException;
+
+ @Override
+ public final void stop() throws HyracksDataException, InterruptedException {
+ try {
+ abort();
+ } finally {
+ if (!done) {
+ synchronized (this) {
+ while (!done) {
+ wait();
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * called from a different thread. This method stops the active node and force the start() call to return
+ *
+ * @throws HyracksDataException
+ * @throws InterruptedException
+ */
+ protected abstract void abort() throws HyracksDataException, InterruptedException;
+
+ @Override
+ public String toString() {
+ return runtimeId.toString();
+ }
+
+ @Override
+ public final void initialize() throws HyracksDataException {
+ activeManager.registerRuntime(this);
+ try {
+ // notify cc that runtime has been registered
+ ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
+ ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED), null);
+ start();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new HyracksDataException(e);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ synchronized (this) {
+ done = true;
+ notifyAll();
+ }
+ }
+ }
+
+ @Override
+ public final void deinitialize() throws HyracksDataException {
+ activeManager.deregisterRuntime(runtimeId);
+ try {
+ ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
+ ActivePartitionMessage.ACTIVE_RUNTIME_DEREGISTERED), null);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public final int getInputArity() {
+ return 0;
+ }
+
+ @Override
+ public final IFrameWriter getInputFrameWriter(int index) {
+ return null;
+ }
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
index 32c5c50..528c220 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
@@ -18,11 +18,20 @@
*/
package org.apache.asterix.active;
-@FunctionalInterface
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
public interface IActiveRuntime {
/**
- * @return the unique runtime id associated with the feedRuntime
+ * @return the unique runtime id associated with the active runtime
*/
- public ActiveRuntimeId getRuntimeId();
+ ActiveRuntimeId getRuntimeId();
+
+ /**
+ * Stops the running activity
+ *
+ * @throws HyracksDataException
+ * @throws InterruptedException
+ */
+ void stop() throws HyracksDataException, InterruptedException;
}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntimeRegistry.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntimeRegistry.java
deleted file mode 100644
index b2c6f8e..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntimeRegistry.java
+++ /dev/null
@@ -1,49 +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.active;
-
-import java.io.IOException;
-
-public interface IActiveRuntimeRegistry {
-
- /**
- * Allows registration of a runtime.
- *
- * @param runtime
- * @throws Exception
- */
- public void registerRuntime(ActiveRuntime runtime);
-
- /**
- * Obtain runtime corresponding to a feedRuntimeId
- *
- * @param runtimeId
- * @return
- */
- public ActiveRuntime getRuntime(ActiveRuntimeId runtimeId);
-
- /**
- * De-register a feed
- *
- * @param feedConnection
- * @throws IOException
- */
- void deregisterRuntime(ActiveRuntimeId runtimeId);
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
similarity index 69%
rename from asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
index 8875647..a6e1788 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
@@ -16,40 +16,40 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.active;
+package org.apache.asterix.active.message;
import java.io.Serializable;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
-import org.apache.hyracks.api.job.JobId;
-public class ActivePartitionMessage extends AbstractApplicationMessage {
+public class ActiveManagerMessage extends AbstractApplicationMessage {
+ public static final byte STOP_ACTIVITY = 0x00;
private static final long serialVersionUID = 1L;
- private final EntityId feedId;
- private final JobId jobId;
+ private final byte kind;
+ private final String src;
private final Serializable payload;
- public ActivePartitionMessage(EntityId feedId, JobId jobId, Serializable payload) {
- this.feedId = feedId;
- this.jobId = jobId;
+ public ActiveManagerMessage(byte kind, String src, Serializable payload) {
+ this.kind = kind;
+ this.src = src;
this.payload = payload;
}
@Override
public ApplicationMessageType getMessageType() {
- return ApplicationMessageType.ACTIVE_ENTITY_MESSAGE;
- }
-
- public EntityId getFeedId() {
- return feedId;
- }
-
- public JobId getJobId() {
- return jobId;
+ return ApplicationMessageType.ACTIVE_MANAGER_MESSAGE;
}
public Serializable getPayload() {
return payload;
}
+
+ public byte getKind() {
+ return kind;
+ }
+
+ public String getSrc() {
+ return src;
+ }
}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
similarity index 62%
copy from asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
copy to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
index 8875647..f5bdf39 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
@@ -16,33 +16,42 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.active;
+package org.apache.asterix.active.message;
import java.io.Serializable;
+import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
import org.apache.hyracks.api.job.JobId;
public class ActivePartitionMessage extends AbstractApplicationMessage {
+ public static final byte ACTIVE_RUNTIME_REGISTERED = 0x00;
+ public static final byte ACTIVE_RUNTIME_DEREGISTERED = 0x01;
private static final long serialVersionUID = 1L;
- private final EntityId feedId;
+ private final ActiveRuntimeId activeRuntimeId;
private final JobId jobId;
private final Serializable payload;
+ private final byte event;
- public ActivePartitionMessage(EntityId feedId, JobId jobId, Serializable payload) {
- this.feedId = feedId;
+ public ActivePartitionMessage(ActiveRuntimeId activeRuntimeId, JobId jobId, byte event) {
+ this(activeRuntimeId, jobId, event, null);
+ }
+
+ public ActivePartitionMessage(ActiveRuntimeId activeRuntimeId, JobId jobId, byte event, Serializable payload) {
+ this.activeRuntimeId = activeRuntimeId;
this.jobId = jobId;
+ this.event = event;
this.payload = payload;
}
@Override
public ApplicationMessageType getMessageType() {
- return ApplicationMessageType.ACTIVE_ENTITY_MESSAGE;
+ return ApplicationMessageType.ACTIVE_ENTITY_TO_CC_MESSAGE;
}
- public EntityId getFeedId() {
- return feedId;
+ public ActiveRuntimeId getActiveRuntimeId() {
+ return activeRuntimeId;
}
public JobId getJobId() {
@@ -52,4 +61,8 @@
public Serializable getPayload() {
return payload;
}
+
+ public byte getEvent() {
+ return event;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
new file mode 100644
index 0000000..ac3bc03
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExtension.java
@@ -0,0 +1,64 @@
+/*
+ * 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.algebra.base;
+
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+/**
+ * An interface for language extensions
+ */
+public interface ILangExtension extends IExtension {
+
+ public enum Language {
+ AQL,
+ SQLPP
+ }
+
+ @Override
+ default ExtensionKind getExtensionKind() {
+ return ExtensionKind.LANG;
+ }
+
+ ILangCompilationProvider getLangCompilationProvider(Language lang);
+
+ //TODO(amoudi/yingyi) this is not a good way to extend re-write rules. introduce rewrite-rule-provider.
+ /**
+ * Called by the compiler when the unnest function is an extension function.
+ * Provides a way to add additional types of datasources
+ *
+ * @param opRef
+ * @param context
+ * @param unnestOp
+ * @param unnestExpr
+ * @param functionCallExpr
+ * @return true if transformation was successful, false otherwise
+ * @throws AlgebricksException
+ */
+ boolean unnestToDataScan(Mutable<ILogicalOperator> opRef, IOptimizationContext context, UnnestOperator unnestOp,
+ ILogicalExpression unnestExpr, AbstractFunctionCallExpression functionCallExpr) throws AlgebricksException;
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionFunctionIdentifier.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionFunctionIdentifier.java
new file mode 100644
index 0000000..ae67337
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionFunctionIdentifier.java
@@ -0,0 +1,78 @@
+/*
+ * 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.algebra.extension;
+
+import org.apache.asterix.common.api.ExtensionId;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * an Identifier for an extension function
+ */
+public class ExtensionFunctionIdentifier extends FunctionIdentifier {
+ //TODO (till): remove this classs
+
+ private static final long serialVersionUID = 1L;
+ private final ExtensionId extensionId;
+
+ /**
+ * Create an identifier for an external function
+ *
+ * @param namespace
+ * @param name
+ * @param extensionId
+ */
+ public ExtensionFunctionIdentifier(String namespace, String name, ExtensionId extensionId) {
+ super(namespace, name);
+ this.extensionId = extensionId;
+ }
+
+ /**
+ * Create an identifier for an external function
+ *
+ * @param namespace
+ * @param name
+ * @param arity
+ * @param extensionId
+ */
+ public ExtensionFunctionIdentifier(String namespace, String name, int arity, ExtensionId extensionId) {
+ super(namespace, name, arity);
+ this.extensionId = extensionId;
+ }
+
+ public ExtensionId getExtensionId() {
+ return extensionId;
+ }
+
+ @Override
+ public int hashCode() {
+ return super.hashCode() + extensionId.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o instanceof ExtensionFunctionIdentifier) {
+ ExtensionFunctionIdentifier oId = (ExtensionFunctionIdentifier) o;
+ return super.equals(o) && extensionId.equals(oId.getExtensionId());
+ }
+ return false;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/IAlgebraExtensionManager.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/IAlgebraExtensionManager.java
new file mode 100644
index 0000000..0a8402c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/IAlgebraExtensionManager.java
@@ -0,0 +1,34 @@
+/*
+ * 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.algebra.extension;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+public interface IAlgebraExtensionManager {
+
+ boolean unnestToDataScan(Mutable<ILogicalOperator> opRef, IOptimizationContext context, UnnestOperator unnestOp,
+ ILogicalExpression unnestExpr, AbstractFunctionCallExpression functionCallExpr) throws AlgebricksException;
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/IExtensionStatement.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/IExtensionStatement.java
new file mode 100644
index 0000000..9e2241f
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/IExtensionStatement.java
@@ -0,0 +1,48 @@
+/*
+ * 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.algebra.extension;
+
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.metadata.declared.AqlMetadataProvider;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+
+/**
+ * An interface that provides an extension mechanism to extend a language with additional statements
+ */
+public interface IExtensionStatement extends Statement {
+
+ @Override
+ default byte getKind() {
+ return Kind.EXTENSION;
+ }
+
+ /**
+ * Called when the {@code IQueryTranslator} encounters an extension statement.
+ * An implementation class should implement the actual processing of the statement in this method.
+ *
+ * @param queryTranslator
+ * @param metadataProvider
+ * @param statementExecutor
+ * @param hcc
+ * @throws Exception
+ */
+ void handle(IStatementExecutor statementExecutor, AqlMetadataProvider metadataProvider,
+ IHyracksClientConnection hcc) throws Exception;
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 720de76..b29f743 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -22,6 +22,7 @@
import java.util.LinkedList;
import java.util.List;
+import org.apache.asterix.algebra.extension.IAlgebraExtensionManager;
import org.apache.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
import org.apache.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
import org.apache.asterix.optimizer.rules.AsterixInlineVariablesRule;
@@ -152,7 +153,9 @@
return autogen;
}
- public static final List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
+ //TODO(amoudi/yingyi): refactor this to use a provider instead of passing the extensionManager
+ public static final List<IAlgebraicRewriteRule> buildNormalizationRuleCollection(
+ IAlgebraExtensionManager algebraExtensionManager) {
List<IAlgebraicRewriteRule> normalization = new LinkedList<>();
normalization.add(new ResolveVariableRule());
normalization.add(new IntroduceUnnestForCollectionToSequenceRule());
@@ -174,7 +177,7 @@
normalization.add(new ExtractCommonExpressionsRule());
normalization.add(new ConstantFoldingRule());
normalization.add(new RemoveRedundantSelectRule());
- normalization.add(new UnnestToDataScanRule());
+ normalization.add(new UnnestToDataScanRule(algebraExtensionManager));
normalization.add(new MetaFunctionToMetaVariableRule());
normalization.add(new FuzzyEqRule());
normalization.add(new SimilarityCheckRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index 605ddb4..1fa7730 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -24,7 +24,7 @@
import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.metadata.declared.AqlDataSource;
import org.apache.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
-import org.apache.asterix.metadata.declared.FeedDataSource;
+import org.apache.asterix.metadata.declared.IMutationDataSource;
import org.apache.asterix.om.constants.AsterixConstantValue;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.IAType;
@@ -86,11 +86,11 @@
LogicalVariable metaVar = dataSource.getMetaVariable(allVars);
LogicalExpressionReferenceTransform currentTransformer = null;
if (dataSource.getDatasourceType() == AqlDataSourceType.FEED) {
- FeedDataSource fds = (FeedDataSource) dataSource;
- if (fds.isChange()) {
+ IMutationDataSource mds = (IMutationDataSource) dataSource;
+ if (mds.isChange()) {
transformers = new ArrayList<>();
- transformers.add(new MetaKeyExpressionReferenceTransform(fds.getPkVars(allVars),
- fds.getKeyAccessExpression()));
+ transformers.add(new MetaKeyExpressionReferenceTransform(mds.getPkVars(allVars),
+ mds.getKeyAccessExpression()));
} else if (metaVar != null) {
transformers = new ArrayList<>();
transformers.add(new MetaKeyToFieldAccessTransform(metaVar));
@@ -98,7 +98,7 @@
}
if (!dataSource.hasMeta() && transformers == null) {
return inputTransfomer;
- };
+ }
if (metaVar != null) {
currentTransformer = new LogicalExpressionReferenceTransform(dataVar, metaVar);
}
@@ -286,7 +286,7 @@
throw new AlgebricksException("Unsupported field name type " + fieldNameType.getTypeTag());
}
IFunctionInfo finfoAccess = FunctionUtil.getFunctionInfo(functionIdentifier);
- ArrayList<Mutable<ILogicalExpression>> argExprs = new ArrayList<Mutable<ILogicalExpression>>(2);
+ ArrayList<Mutable<ILogicalExpression>> argExprs = new ArrayList<>(2);
argExprs.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
argExprs.add(new MutableObject<>(fieldNameExpression));
exprRef.setValue(new ScalarFunctionCallExpression(finfoAccess, argExprs));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
index bb382f0..17dec7c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -18,11 +18,10 @@
*/
package org.apache.asterix.optimizer.rules;
-import static org.apache.asterix.om.util.ConstantExpressionUtil.getStringArgument;
-
import java.util.ArrayList;
import java.util.List;
+import org.apache.asterix.algebra.extension.IAlgebraExtensionManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.external.feed.watch.FeedActivity.FeedActivityDetails;
import org.apache.asterix.external.util.ExternalDataUtils;
@@ -45,6 +44,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.ConstantExpressionUtil;
import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.asterix.translator.util.PlanTranslationUtil;
import org.apache.commons.lang3.mutable.Mutable;
@@ -68,6 +68,11 @@
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
+ private final IAlgebraExtensionManager algebraExtensionManager;
+
+ public UnnestToDataScanRule(IAlgebraExtensionManager algebraExtensionManager) {
+ this.algebraExtensionManager = algebraExtensionManager;
+ }
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -150,19 +155,17 @@
EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(scan, variables, recordType,
metaRecordType, dataset, context);
return true;
- }
-
- if (fid.equals(AsterixBuiltinFunctions.FEED_COLLECT)) {
+ } else if (fid.equals(AsterixBuiltinFunctions.FEED_COLLECT)) {
if (unnest.getPositionalVariable() != null) {
throw new AlgebricksException("No positional variables are allowed over feeds.");
}
- String dataverse = getStringArgument(f, 0);
- String sourceFeedName = getStringArgument(f, 1);
- String getTargetFeed = getStringArgument(f, 2);
- String subscriptionLocation = getStringArgument(f, 3);
- String targetDataset = getStringArgument(f, 4);
- String outputType = getStringArgument(f, 5);
+ String dataverse = ConstantExpressionUtil.getStringArgument(f, 0);
+ String sourceFeedName = ConstantExpressionUtil.getStringArgument(f, 1);
+ String getTargetFeed = ConstantExpressionUtil.getStringArgument(f, 2);
+ String subscriptionLocation = ConstantExpressionUtil.getStringArgument(f, 3);
+ String targetDataset = ConstantExpressionUtil.getStringArgument(f, 4);
+ String outputType = ConstantExpressionUtil.getStringArgument(f, 5);
AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
@@ -190,10 +193,7 @@
}
// Does it produce pk?
if (ds.isChange()) {
- int numOfPKs = ds.getPkTypes().size();
- for (int i = 0; i < numOfPKs; i++) {
- feedDataScanOutputVariables.addAll(pkVars);
- }
+ feedDataScanOutputVariables.addAll(pkVars);
}
DataSourceScanOperator scan = new DataSourceScanOperator(feedDataScanOutputVariables, ds);
@@ -202,8 +202,9 @@
opRef.setValue(scan);
context.computeAndSetTypeEnvironmentForOperator(scan);
return true;
+ } else if (algebraExtensionManager != null) {
+ return algebraExtensionManager.unnestToDataScan(opRef, context, unnest, unnestExpr, f);
}
-
}
return false;
@@ -297,6 +298,6 @@
dataverseName = datasetNameComponents[0];
datasetName = datasetNameComponents[1];
}
- return new Pair<String, String>(dataverseName, datasetName);
+ return new Pair<>(dataverseName, datasetName);
}
}
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index b3e9e09..16ac80d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -72,15 +72,15 @@
private AqlMetadataProvider metadataProvider;
// Function Identifier sets that retain the original field variable through each function's arguments
- private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName = ImmutableSet.of(
- AsterixBuiltinFunctions.WORD_TOKENS, AsterixBuiltinFunctions.GRAM_TOKENS, AsterixBuiltinFunctions.SUBSTRING,
- AsterixBuiltinFunctions.SUBSTRING_BEFORE, AsterixBuiltinFunctions.SUBSTRING_AFTER,
- AsterixBuiltinFunctions.CREATE_POLYGON, AsterixBuiltinFunctions.CREATE_MBR,
- AsterixBuiltinFunctions.CREATE_RECTANGLE, AsterixBuiltinFunctions.CREATE_CIRCLE,
- AsterixBuiltinFunctions.CREATE_LINE, AsterixBuiltinFunctions.CREATE_POINT,
- AsterixBuiltinFunctions.NUMERIC_ADD, AsterixBuiltinFunctions.NUMERIC_SUBTRACT,
- AsterixBuiltinFunctions.NUMERIC_MULTIPLY, AsterixBuiltinFunctions.NUMERIC_DIVIDE,
- AsterixBuiltinFunctions.NUMERIC_MOD);
+ private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
+ ImmutableSet.of(AsterixBuiltinFunctions.WORD_TOKENS, AsterixBuiltinFunctions.GRAM_TOKENS,
+ AsterixBuiltinFunctions.SUBSTRING, AsterixBuiltinFunctions.SUBSTRING_BEFORE,
+ AsterixBuiltinFunctions.SUBSTRING_AFTER, AsterixBuiltinFunctions.CREATE_POLYGON,
+ AsterixBuiltinFunctions.CREATE_MBR, AsterixBuiltinFunctions.CREATE_RECTANGLE,
+ AsterixBuiltinFunctions.CREATE_CIRCLE, AsterixBuiltinFunctions.CREATE_LINE,
+ AsterixBuiltinFunctions.CREATE_POINT, AsterixBuiltinFunctions.NUMERIC_ADD,
+ AsterixBuiltinFunctions.NUMERIC_SUBTRACT, AsterixBuiltinFunctions.NUMERIC_MULTIPLY,
+ AsterixBuiltinFunctions.NUMERIC_DIVIDE, AsterixBuiltinFunctions.NUMERIC_MOD);
public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
@@ -145,15 +145,15 @@
return list.isEmpty() ? null : list.get(0);
}
- protected List<Pair<IAccessMethod, Index>> chooseAllIndex(
- Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+ protected List<Pair<IAccessMethod, Index>>
+ chooseAllIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
List<Pair<IAccessMethod, Index>> result = new ArrayList<Pair<IAccessMethod, Index>>();
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
while (amIt.hasNext()) {
Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
- Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = analysisCtx.indexExprsAndVars.entrySet()
- .iterator();
+ Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt =
+ analysisCtx.indexExprsAndVars.entrySet().iterator();
while (indexIt.hasNext()) {
Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
// To avoid a case where the chosen access method and a chosen
@@ -167,11 +167,11 @@
// LENGTH_PARTITIONED_NGRAM_INVIX]
IAccessMethod chosenAccessMethod = amEntry.getKey();
Index chosenIndex = indexEntry.getKey();
- boolean isKeywordOrNgramIndexChosen = chosenIndex
- .getIndexType() == IndexType.LENGTH_PARTITIONED_WORD_INVIX
- || chosenIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
- || chosenIndex.getIndexType() == IndexType.SINGLE_PARTITION_WORD_INVIX
- || chosenIndex.getIndexType() == IndexType.SINGLE_PARTITION_NGRAM_INVIX;
+ boolean isKeywordOrNgramIndexChosen =
+ chosenIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || chosenIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
+ || chosenIndex.getIndexType() == IndexType.SINGLE_PARTITION_WORD_INVIX
+ || chosenIndex.getIndexType() == IndexType.SINGLE_PARTITION_NGRAM_INVIX;
if ((chosenAccessMethod == BTreeAccessMethod.INSTANCE && chosenIndex.getIndexType() == IndexType.BTREE)
|| (chosenAccessMethod == RTreeAccessMethod.INSTANCE
@@ -196,8 +196,8 @@
*/
public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx,
IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
- Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt = analysisCtx.indexExprsAndVars
- .entrySet().iterator();
+ Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt =
+ analysisCtx.indexExprsAndVars.entrySet().iterator();
// Used to keep track of matched expressions (added for prefix search)
int numMatchedKeys = 0;
ArrayList<Integer> matchedExpressions = new ArrayList<Integer>();
@@ -293,8 +293,8 @@
// Check if any field name in the optFuncExpr matches.
if (optFuncExpr.findFieldName(keyField) != -1) {
- foundKeyField = typeMatch
- && optFuncExpr.getOperatorSubTree(exprAndVarIdx.second).hasDataSourceScan();
+ foundKeyField =
+ typeMatch && optFuncExpr.getOperatorSubTree(exprAndVarIdx.second).hasDataSourceScan();
if (foundKeyField) {
matchedExpressions.add(exprAndVarIdx.first);
numMatchedKeys++;
@@ -369,8 +369,8 @@
continue;
}
AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
- boolean matchFound = analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context,
- typeEnvironment);
+ boolean matchFound =
+ analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
found = found || matchFound;
}
return found;
@@ -441,14 +441,15 @@
// fieldName
for (Index index : datasetIndexes) {
// Need to also verify the index is pending no op
- if (index.getKeyFieldNames().contains(fieldName) && index.getPendingOp() == IMetadataEntity.PENDING_NO_OP) {
+ if (index.getKeyFieldNames().contains(fieldName)
+ && index.getPendingOp() == IMetadataEntity.PENDING_NO_OP) {
indexCandidates.add(index);
if (optFuncExpr.getFieldType(varIdx) == BuiltinType.AMISSING
|| optFuncExpr.getFieldType(varIdx) == BuiltinType.ANY) {
optFuncExpr.setFieldType(varIdx,
index.getKeyFieldTypes().get(index.getKeyFieldNames().indexOf(fieldName)));
}
- analysisCtx.addIndexExpr(matchedSubTree.dataset, index, matchedFuncExprIndex, varIdx);
+ analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
}
}
// No index candidates for fieldName.
@@ -464,11 +465,11 @@
List<Index> datasetIndexes = new ArrayList<Index>();
LogicalVariable datasetMetaVar = null;
LogicalVariable datasetRecordVar = null;
- if (subTree.dataSourceType != DataSourceType.COLLECTION_SCAN) {
- datasetIndexes = metadataProvider.getDatasetIndexes(subTree.dataset.getDataverseName(),
- subTree.dataset.getDatasetName());
+ if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
+ datasetIndexes = metadataProvider.getDatasetIndexes(subTree.getDataset().getDataverseName(),
+ subTree.getDataset().getDatasetName());
List<LogicalVariable> datasetVars = subTree.getDataSourceVariables();
- if (subTree.dataset.hasMetaPart()) {
+ if (subTree.getDataset().hasMetaPart()) {
datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
datasetRecordVar = datasetVars.get(datasetVars.size() - 2);
} else {
@@ -477,9 +478,9 @@
}
for (IOptimizableFuncExpr optFuncExpr : analysisCtx.matchedFuncExprs) {
// Try to match variables from optFuncExpr to assigns or unnests.
- for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.assignsAndUnnests
+ for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
.size(); assignOrUnnestIndex++) {
- AbstractLogicalOperator op = subTree.assignsAndUnnests.get(assignOrUnnestIndex);
+ AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
analyzeAssignOp((AssignOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetRecordVar,
datasetMetaVar, context, datasetIndexes, optFuncExprIndex, analysisCtx);
@@ -501,7 +502,7 @@
if (subTree.hasIxJoinOuterAdditionalDataSource()) {
additionalDsVarList = new ArrayList<LogicalVariable>();
- for (int i = 0; i < subTree.ixJoinOuterAdditionalDataSourceRefs.size(); i++) {
+ for (int i = 0; i < subTree.getIxJoinOuterAdditionalDataSourceRefs().size(); i++) {
additionalDsVarList.addAll(subTree.getIxJoinOuterAdditionalDataSourceVariables(i));
}
@@ -529,18 +530,18 @@
// Remember matching subtree.
optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
List<String> fieldName = null;
- if (subTree.dataSourceType == DataSourceType.COLLECTION_SCAN) {
+ if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
} else {
- fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.recordType,
+ fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
- datasetRecordVar, subTree.metaRecordType, datasetMetaVar);
+ datasetRecordVar, subTree.getMetaRecordType(), datasetMetaVar);
if (fieldName == null) {
return;
}
}
- IAType fieldType = (IAType) context.getOutputTypeEnvironment(unnestOp)
- .getType(optFuncExpr.getLogicalExpr(funcVarIndex));
+ IAType fieldType =
+ (IAType) context.getOutputTypeEnvironment(unnestOp).getType(optFuncExpr.getLogicalExpr(funcVarIndex));
// Set the fieldName in the corresponding matched function
// expression.
optFuncExpr.setFieldName(funcVarIndex, fieldName);
@@ -570,10 +571,11 @@
// Remember matching subtree.
optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
- List<String> fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
- subTree.recordType, optVarIndex,
- optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), datasetRecordVar,
- subTree.metaRecordType, datasetMetaVar);
+ List<String> fieldName =
+ getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
+ subTree.getRecordType(), optVarIndex,
+ optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(),
+ datasetRecordVar, subTree.getMetaRecordType(), datasetMetaVar);
if (fieldName == null) {
continue;
}
@@ -609,25 +611,27 @@
List<List<String>> subTreePKs = null;
if (!fromAdditionalDataSource) {
- subTreePKs = DatasetUtils.getPartitioningKeys(subTree.dataset);
+ subTreePKs = DatasetUtils.getPartitioningKeys(subTree.getDataset());
// Check whether this variable is PK, not a record variable.
if (varIndex <= subTreePKs.size() - 1) {
fieldName = subTreePKs.get(varIndex);
- fieldType = (IAType) context.getOutputTypeEnvironment(subTree.dataSourceRef.getValue())
- .getVarType(var);
+ fieldType =
+ (IAType) context.getOutputTypeEnvironment(
+ subTree.getDataSourceRef().getValue()).getVarType(var);
}
} else {
// Need to check additional dataset one by one
- for (int i = 0; i < subTree.ixJoinOuterAdditionalDatasets.size(); i++) {
- if (subTree.ixJoinOuterAdditionalDatasets.get(i) != null) {
- subTreePKs = DatasetUtils.getPartitioningKeys(subTree.ixJoinOuterAdditionalDatasets.get(i));
+ for (int i = 0; i < subTree.getIxJoinOuterAdditionalDatasets().size(); i++) {
+ if (subTree.getIxJoinOuterAdditionalDatasets().get(i) != null) {
+ subTreePKs = DatasetUtils.getPartitioningKeys(
+ subTree.getIxJoinOuterAdditionalDatasets().get(i));
// Check whether this variable is PK, not a record variable.
if (subTreePKs.contains(var) && varIndex <= subTreePKs.size() - 1) {
fieldName = subTreePKs.get(varIndex);
fieldType = (IAType) context
.getOutputTypeEnvironment(
- subTree.ixJoinOuterAdditionalDataSourceRefs.get(i).getValue())
+ subTree.getIxJoinOuterAdditionalDataSourceRefs().get(i).getValue())
.getVarType(var);
break;
}
@@ -651,7 +655,7 @@
private void setTypeTag(IOptimizationContext context, OptimizableOperatorSubTree subTree,
IOptimizableFuncExpr optFuncExpr, int funcVarIndex) throws AlgebricksException {
// Set the typeTag if the type is not null
- IAType type = (IAType) context.getOutputTypeEnvironment(subTree.root)
+ IAType type = (IAType) context.getOutputTypeEnvironment(subTree.getRoot())
.getVarType(optFuncExpr.getLogicalVar(funcVarIndex));
optFuncExpr.setFieldType(funcVarIndex, type);
}
@@ -663,14 +667,15 @@
*
* @throws AlgebricksException
*/
- protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
- int opIndex, int assignVarIndex, ARecordType recordType, int funcVarIndex,
- ILogicalExpression parentFuncExpr, LogicalVariable recordVar, ARecordType metaType, LogicalVariable metaVar)
+ protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr,
+ OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
+ int funcVarIndex, ILogicalExpression parentFuncExpr, LogicalVariable recordVar,
+ ARecordType metaType, LogicalVariable metaVar)
throws AlgebricksException {
// Get expression corresponding to opVar at varIndex.
AbstractLogicalExpression expr = null;
AbstractFunctionCallExpression childFuncExpr = null;
- AbstractLogicalOperator op = subTree.assignsAndUnnests.get(opIndex);
+ AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(opIndex);
if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
AssignOperator assignOp = (AssignOperator) op;
expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
@@ -718,8 +723,8 @@
return null;
}
ConstantExpression constExpr = (ConstantExpression) nameArg;
- AOrderedList orderedNestedFieldName = (AOrderedList) ((AsterixConstantValue) constExpr.getValue())
- .getObject();
+ AOrderedList orderedNestedFieldName =
+ (AOrderedList) ((AsterixConstantValue) constExpr.getValue()).getObject();
nestedAccessFieldName = new ArrayList<String>();
for (int i = 0; i < orderedNestedFieldName.size(); i++) {
nestedAccessFieldName.add(((AString) orderedNestedFieldName.getItem(i)).getStringValue());
@@ -728,14 +733,14 @@
isByName = true;
}
if (isFieldAccess) {
- LogicalVariable sourceVar = ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue())
- .getVariableReference();
+ LogicalVariable sourceVar =
+ ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
int[] assignAndExpressionIndexes = null;
//go forward through nested assigns until you find the relevant one
- for (int i = opIndex + 1; i < subTree.assignsAndUnnests.size(); i++) {
- AbstractLogicalOperator subOp = subTree.assignsAndUnnests.get(i);
+ for (int i = opIndex + 1; i < subTree.getAssignsAndUnnests().size(); i++) {
+ AbstractLogicalOperator subOp = subTree.getAssignsAndUnnests().get(i);
List<LogicalVariable> varList;
if (subOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
@@ -822,9 +827,9 @@
LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
// We look for the assign or unnest operator that produces curVar below
// the current operator
- for (int assignOrUnnestIndex = opIndex + 1; assignOrUnnestIndex < subTree.assignsAndUnnests
+ for (int assignOrUnnestIndex = opIndex + 1; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
.size(); assignOrUnnestIndex++) {
- AbstractLogicalOperator curOp = subTree.assignsAndUnnests.get(assignOrUnnestIndex);
+ AbstractLogicalOperator curOp = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
if (curOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
AssignOperator assignOp = (AssignOperator) curOp;
List<LogicalVariable> varList = assignOp.getVariables();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 4dc7bf4..eb7d3a4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -22,6 +22,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
+import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@@ -80,30 +81,28 @@
EQUAL
}
- // TODO: There is some redundancy here, since these are listed in AlgebricksBuiltinFunctions as well.
- private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
+ private static final List<FunctionIdentifier> FUNC_IDENTIFIERS =
+ Collections.unmodifiableList(Arrays.asList(
+ AlgebricksBuiltinFunctions.EQ,
+ AlgebricksBuiltinFunctions.LE,
+ AlgebricksBuiltinFunctions.GE,
+ AlgebricksBuiltinFunctions.LT,
+ AlgebricksBuiltinFunctions.GT));
- static {
- funcIdents.add(AlgebricksBuiltinFunctions.EQ);
- funcIdents.add(AlgebricksBuiltinFunctions.LE);
- funcIdents.add(AlgebricksBuiltinFunctions.GE);
- funcIdents.add(AlgebricksBuiltinFunctions.LT);
- funcIdents.add(AlgebricksBuiltinFunctions.GT);
- }
-
- public static BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
+ public static final BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
@Override
public List<FunctionIdentifier> getOptimizableFunctions() {
- return funcIdents;
+ return FUNC_IDENTIFIERS;
}
@Override
public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr,
List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
- boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx, context,
- typeEnvironment);
+ boolean matches =
+ AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(
+ funcExpr, analysisCtx, context, typeEnvironment);
if (!matches) {
matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVars(funcExpr, analysisCtx);
}
@@ -131,8 +130,8 @@
if (primaryIndexUnnestOp == null) {
return false;
}
- Mutable<ILogicalOperator> opRef = (subTree.assignsAndUnnestsRefs.isEmpty()) ? null
- : subTree.assignsAndUnnestsRefs.get(0);
+ Mutable<ILogicalOperator> opRef =
+ subTree.getAssignsAndUnnestsRefs().isEmpty() ? null : subTree.getAssignsAndUnnestsRefs().get(0);
ILogicalOperator op = null;
if (opRef != null) {
op = opRef.getValue();
@@ -141,7 +140,7 @@
if (conditionRef.getValue() != null) {
select.getInputs().clear();
if (op != null) {
- subTree.dataSourceRef.setValue(primaryIndexUnnestOp);
+ subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
select.getInputs().add(new MutableObject<ILogicalOperator>(op));
} else {
select.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestOp));
@@ -149,7 +148,7 @@
} else {
((AbstractLogicalOperator) primaryIndexUnnestOp).setExecutionMode(ExecutionMode.PARTITIONED);
if (op != null) {
- subTree.dataSourceRef.setValue(primaryIndexUnnestOp);
+ subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
selectRef.setValue(op);
} else {
selectRef.setValue(primaryIndexUnnestOp);
@@ -165,15 +164,16 @@
boolean hasGroupBy) throws AlgebricksException {
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
- // Determine if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
+ // Determine if the index is applicable on the left or right side
+ // (if both, we arbitrarily prefer the left side).
Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
- OptimizableOperatorSubTree indexSubTree = null;
- OptimizableOperatorSubTree probeSubTree = null;
+ OptimizableOperatorSubTree indexSubTree;
+ OptimizableOperatorSubTree probeSubTree;
// We assume that the left subtree is the outer branch and the right subtree is the inner branch.
// This assumption holds true since we only use an index from the right subtree.
// The following is just a sanity check.
if (rightSubTree.hasDataSourceScan()
- && dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+ && dataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
indexSubTree = rightSubTree;
probeSubTree = leftSubTree;
} else {
@@ -199,16 +199,16 @@
}
// If there are conditions left, add a new select operator on top.
- indexSubTree.dataSourceRef.setValue(primaryIndexUnnestOp);
+ indexSubTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
if (conditionRef.getValue() != null) {
SelectOperator topSelect = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
- topSelect.getInputs().add(indexSubTree.rootRef);
+ topSelect.getInputs().add(indexSubTree.getRootRef());
topSelect.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(topSelect);
// Replace the original join with the new subtree rooted at the select op.
joinRef.setValue(topSelect);
} else {
- joinRef.setValue(indexSubTree.rootRef.getValue());
+ joinRef.setValue(indexSubTree.getRootRef().getValue());
}
return true;
}
@@ -216,19 +216,20 @@
@Override
public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
- AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
- IOptimizationContext context) throws AlgebricksException {
- Dataset dataset = indexSubTree.dataset;
- ARecordType recordType = indexSubTree.recordType;
- ARecordType metaRecordType = indexSubTree.metaRecordType;
+ AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull,
+ boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
+ Dataset dataset = indexSubTree.getDataset();
+ ARecordType recordType = indexSubTree.getRecordType();
+ ARecordType metaRecordType = indexSubTree.getMetaRecordType();
// we made sure indexSubTree has datasource scan
- AbstractDataSourceOperator dataSourceOp = (AbstractDataSourceOperator) indexSubTree.dataSourceRef.getValue();
+ AbstractDataSourceOperator dataSourceOp =
+ (AbstractDataSourceOperator) indexSubTree.getDataSourceRef().getValue();
List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.indexExprsAndVars.get(chosenIndex);
List<IOptimizableFuncExpr> matchedFuncExprs = analysisCtx.matchedFuncExprs;
int numSecondaryKeys = analysisCtx.indexNumMatchedKeys.get(chosenIndex);
// List of function expressions that will be replaced by the secondary-index search.
// These func exprs will be removed from the select condition at the very end of this method.
- Set<ILogicalExpression> replacedFuncExprs = new HashSet<ILogicalExpression>();
+ Set<ILogicalExpression> replacedFuncExprs = new HashSet<>();
// Info on high and low keys for the BTree search predicate.
ILogicalExpression[] lowKeyExprs = new ILogicalExpression[numSecondaryKeys];
@@ -240,12 +241,12 @@
ILogicalExpression[] constantAtRuntimeExpressions = new ILogicalExpression[numSecondaryKeys];
LogicalVariable[] constAtRuntimeExprVars = new LogicalVariable[numSecondaryKeys];
- // TODO: For now we don't do any sophisticated analysis of the func exprs to come up with "the best" range predicate.
- // If we can't figure out how to integrate a certain funcExpr into the current predicate, we just bail by setting this flag.
+ /* TODO: For now we don't do any sophisticated analysis of the func exprs to come up with "the best" range
+ * predicate. If we can't figure out how to integrate a certain funcExpr into the current predicate,
+ * we just bail by setting this flag.*/
boolean couldntFigureOut = false;
boolean doneWithExprs = false;
boolean isEqCondition = false;
- // TODO: For now don't consider prefix searches.
BitSet setLowKeys = new BitSet(numSecondaryKeys);
BitSet setHighKeys = new BitSet(numSecondaryKeys);
// Go through the func exprs listed as optimizable by the chosen index,
@@ -253,24 +254,22 @@
// checks whether a type casting happened from a real (FLOAT, DOUBLE) value to an INT value
// since we have a round issues when dealing with LT(<) OR GT(>) operator.
- boolean realTypeConvertedToIntegerType = false;
+ boolean realTypeConvertedToIntegerType;
for (Pair<Integer, Integer> exprIndex : exprAndVarList) {
// Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
IOptimizableFuncExpr optFuncExpr = matchedFuncExprs.get(exprIndex.first);
int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
- if (keyPos < 0) {
- if (optFuncExpr.getNumLogicalVars() > 1) {
- // If we are optimizing a join, the matching field may be the second field name.
- keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
- }
+ if (keyPos < 0 && optFuncExpr.getNumLogicalVars() > 1) {
+ // If we are optimizing a join, the matching field may be the second field name.
+ keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
}
if (keyPos < 0) {
throw new AlgebricksException(
"Could not match optimizable function expression to any index field name.");
}
- Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr = AccessMethodUtils.createSearchKeyExpr(optFuncExpr,
- indexSubTree, probeSubTree);
+ Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr =
+ AccessMethodUtils.createSearchKeyExpr(optFuncExpr, indexSubTree, probeSubTree);
ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
constantAtRuntimeExpressions[keyPos] = searchKeyExpr;
@@ -288,7 +287,7 @@
//
// for $emp in dataset empDataset
// where $emp.age > double("2.3") and $emp.age < double("3.3")
- // return $emp.id;
+ // return $emp.id
//
// It should generate a result if there is a tuple that satisfies the condition, which is 3,
// however, it does not generate the desired result since finding candidates
@@ -316,7 +315,8 @@
setHighKeys.set(keyPos);
isEqCondition = true;
} else {
- // Has already been set to the identical values. When optimizing join we may encounter the same optimizable expression twice
+ // Has already been set to the identical values.
+ // When optimizing join we may encounter the same optimizable expression twice
// (once from analyzing each side of the join)
if (lowKeyLimits[keyPos] == limit && lowKeyInclusive[keyPos] == true
&& lowKeyExprs[keyPos].equals(searchKeyExpr) && highKeyLimits[keyPos] == limit
@@ -339,7 +339,8 @@
highKeyExprs[keyPos] = searchKeyExpr;
highKeyInclusive[keyPos] = false;
} else {
- // Has already been set to the identical values. When optimizing join we may encounter the same optimizable expression twice
+ // Has already been set to the identical values. When optimizing join we may encounter the
+ // same optimizable expression twice
// (once from analyzing each side of the join)
if (highKeyLimits[keyPos] == limit && highKeyInclusive[keyPos] == false
&& highKeyExprs[keyPos].equals(searchKeyExpr)) {
@@ -356,7 +357,8 @@
highKeyExprs[keyPos] = searchKeyExpr;
highKeyInclusive[keyPos] = true;
} else {
- // Has already been set to the identical values. When optimizing join we may encounter the same optimizable expression twice
+ // Has already been set to the identical values. When optimizing join we may encounter the
+ // same optimizable expression twice
// (once from analyzing each side of the join)
if (highKeyLimits[keyPos] == limit && highKeyInclusive[keyPos] == true
&& highKeyExprs[keyPos].equals(searchKeyExpr)) {
@@ -373,7 +375,8 @@
lowKeyExprs[keyPos] = searchKeyExpr;
lowKeyInclusive[keyPos] = false;
} else {
- // Has already been set to the identical values. When optimizing join we may encounter the same optimizable expression twice
+ // Has already been set to the identical values. When optimizing join we may encounter the
+ // same optimizable expression twice
// (once from analyzing each side of the join)
if (lowKeyLimits[keyPos] == limit && lowKeyInclusive[keyPos] == false
&& lowKeyExprs[keyPos].equals(searchKeyExpr)) {
@@ -390,7 +393,8 @@
lowKeyExprs[keyPos] = searchKeyExpr;
lowKeyInclusive[keyPos] = true;
} else {
- // Has already been set to the identical values. When optimizing join we may encounter the same optimizable expression twice
+ // Has already been set to the identical values. When optimizing join we may encounter the
+ // same optimizable expression twice
// (once from analyzing each side of the join)
if (lowKeyLimits[keyPos] == limit && lowKeyInclusive[keyPos] == true
&& lowKeyExprs[keyPos].equals(searchKeyExpr)) {
@@ -417,7 +421,8 @@
return null;
}
- // If the select condition contains mixed open/closed intervals on multiple keys, then we make all intervals closed to obtain a superset of answers and leave the original selection in place.
+ // If the select condition contains mixed open/closed intervals on multiple keys, then we make all intervals
+ // closed to obtain a superset of answers and leave the original selection in place.
boolean primaryIndexPostProccessingIsNeeded = false;
for (int i = 1; i < numSecondaryKeys; ++i) {
if (lowKeyInclusive[i] != lowKeyInclusive[0]) {
@@ -450,7 +455,8 @@
highKeyInclusive[0] = true;
}
- // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
+ // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index
+ // search.
// List of variables for the assign.
ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
// List of variables and expressions for the assign.
@@ -480,7 +486,7 @@
inputOp = assignConstantSearchKeys;
} else {
// All index search keys are variables.
- inputOp = probeSubTree.root;
+ inputOp = probeSubTree.getRoot();
}
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
@@ -494,7 +500,7 @@
// External dataset
UnnestMapOperator externalDataAccessOp = AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp,
dataset, recordType, secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull);
- indexSubTree.dataSourceRef.setValue(externalDataAccessOp);
+ indexSubTree.getDataSourceRef().setValue(externalDataAccessOp);
return externalDataAccessOp;
} else if (!isPrimaryIndex) {
primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceOp, dataset, recordType,
@@ -542,8 +548,8 @@
// An index search is expressed as an unnest-map over an
// index-search function.
IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
- UnnestingFunctionCallExpression primaryIndexSearchFunc = new UnnestingFunctionCallExpression(
- primaryIndexSearch, primaryIndexFuncArgs);
+ UnnestingFunctionCallExpression primaryIndexSearchFunc =
+ new UnnestingFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
primaryIndexSearchFunc.setReturnsUniqueValues(true);
if (!leftOuterUnnestMapRequired) {
primaryIndexUnnestOp = new UnnestMapOperator(scanVariables,
@@ -651,8 +657,8 @@
}
private LimitType getLimitType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree probeSubTree) {
- ComparisonKind ck = AlgebricksBuiltinFunctions
- .getComparisonType(optFuncExpr.getFuncExpr().getFunctionIdentifier());
+ ComparisonKind ck =
+ AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr().getFunctionIdentifier());
LimitType limit = null;
switch (ck) {
case EQ: {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index e1cda09..ff4d219 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -108,11 +108,11 @@
boolean matchInLeftSubTree = false;
boolean matchInRightSubTree = false;
if (leftSubTree.hasDataSource()) {
- matchInLeftSubTree = analyzeCondition(joinCond, leftSubTree.assignsAndUnnests, analyzedAMs, context,
+ matchInLeftSubTree = analyzeCondition(joinCond, leftSubTree.getAssignsAndUnnests(), analyzedAMs, context,
typeEnvironment);
}
if (rightSubTree.hasDataSource()) {
- matchInRightSubTree = analyzeCondition(joinCond, rightSubTree.assignsAndUnnests, analyzedAMs, context,
+ matchInRightSubTree = analyzeCondition(joinCond, rightSubTree.getAssignsAndUnnests(), analyzedAMs, context,
typeEnvironment);
}
if (!matchInLeftSubTree && !matchInRightSubTree) {
@@ -178,8 +178,8 @@
*/
protected void removeIndexCandidatesFromOuterBranch(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
String innerDataset = null;
- if (rightSubTree.dataset != null) {
- innerDataset = rightSubTree.dataset.getDatasetName();
+ if (rightSubTree.getDataset() != null) {
+ innerDataset = rightSubTree.getDataset().getDatasetName();
}
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 45c5e34..0e54640 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -23,6 +23,7 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.metadata.declared.AqlDataSource;
+import org.apache.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.metadata.declared.DatasetDataSource;
import org.apache.asterix.metadata.entities.Dataset;
@@ -102,7 +103,7 @@
List<Index> datasetIndexes = ((AqlMetadataProvider) context.getMetadataProvider())
.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
- List<IOptimizableFuncExpr> optFuncExprs = new ArrayList<IOptimizableFuncExpr>();
+ List<IOptimizableFuncExpr> optFuncExprs = new ArrayList<>();
for (int i = 0; i < analysisCtx.matchedFuncExprs.size(); i++) {
IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(i);
@@ -123,8 +124,8 @@
private AssignOperator createAssignOperator(List<IOptimizableFuncExpr> optFuncExprs,
List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOptimizationContext context) {
- List<LogicalVariable> assignKeyVarList = new ArrayList<LogicalVariable>();
- List<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+ List<LogicalVariable> assignKeyVarList = new ArrayList<>();
+ List<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<>();
for (IOptimizableFuncExpr optFuncExpr : optFuncExprs) {
ComparisonKind ck = AlgebricksBuiltinFunctions
@@ -154,15 +155,15 @@
DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
if (dataset.getDatasetName().compareTo(((DatasetDataSource) ds).getDataset().getDatasetName()) == 0) {
- List<LogicalVariable> minFilterVars = new ArrayList<LogicalVariable>();
- List<LogicalVariable> maxFilterVars = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> minFilterVars = new ArrayList<>();
+ List<LogicalVariable> maxFilterVars = new ArrayList<>();
AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context);
dataSourceScanOp.setMinFilterVars(minFilterVars);
dataSourceScanOp.setMaxFilterVars(maxFilterVars);
- List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();;
+ List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<>();
for (LogicalVariable var : assignOp.getVariables()) {
additionalFilteringExpressions
.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
@@ -186,8 +187,8 @@
AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
jobGenParams.readFromFuncArgs(f.getArguments());
if (dataset.getDatasetName().compareTo(jobGenParams.datasetName) == 0) {
- List<LogicalVariable> minFilterVars = new ArrayList<LogicalVariable>();
- List<LogicalVariable> maxFilterVars = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> minFilterVars = new ArrayList<>();
+ List<LogicalVariable> maxFilterVars = new ArrayList<>();
AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars,
context);
@@ -195,7 +196,7 @@
unnestMapOp.setMinFilterVars(minFilterVars);
unnestMapOp.setMaxFilterVars(maxFilterVars);
- List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();;
+ List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<>();
for (LogicalVariable var : assignOp.getVariables()) {
additionalFilteringExpressions
.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
@@ -220,6 +221,9 @@
if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
+ if (ds.getDatasourceType() != AqlDataSourceType.INTERNAL_DATASET) {
+ return null;
+ }
return ((DatasetDataSource) ds).getDataset();
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
@@ -227,8 +231,8 @@
if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
FunctionIdentifier fid = f.getFunctionIdentifier();
- String dataverseName = null;
- String datasetName = null;
+ String dataverseName;
+ String datasetName;
if (AsterixBuiltinFunctions.EXTERNAL_LOOKUP.equals(fid)) {
dataverseName = AccessMethodUtils.getStringConstant(f.getArguments().get(0));
datasetName = AccessMethodUtils.getStringConstant(f.getArguments().get(1));
@@ -299,7 +303,7 @@
private boolean findMacthedExprFieldName(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
Dataset dataset, ARecordType recType, List<Index> datasetIndexes, IOptimizationContext context)
- throws AlgebricksException {
+ throws AlgebricksException {
AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
while (descendantOp != null) {
if (descendantOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
@@ -346,7 +350,7 @@
continue;
}
- String indexName = null;
+ String indexName;
Index index = null;
ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -408,10 +412,10 @@
|| funcIdent == AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
//get the variable from here. Figure out which input it came from. Go to that input!!!
- ArrayList<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> usedVars = new ArrayList<>();
expr.getUsedVariables(usedVars);
LogicalVariable usedVar = usedVars.get(0);
- List<String> returnList = new ArrayList<String>();
+ List<String> returnList = new ArrayList<>();
//Find the input that it came from
for (int varCheck = 0; varCheck < op.getInputs().size(); varCheck++) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 5b9a9b6..2464c06 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -106,7 +106,7 @@
// Analyze select condition.
Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new TreeMap<IAccessMethod, AccessMethodAnalysisContext>();
- if (!analyzeCondition(selectCond, subTree.assignsAndUnnests, analyzedAMs, context, typeEnvironment)) {
+ if (!analyzeCondition(selectCond, subTree.getAssignsAndUnnests(), analyzedAMs, context, typeEnvironment)) {
return false;
}
@@ -164,7 +164,7 @@
}
ILogicalOperator primaryUnnest = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
- subTree.dataSourceRef.setValue(primaryUnnest);
+ subTree.getDataSourceRef().setValue(primaryUnnest);
return primaryUnnest != null;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 58cc32e..1052a72 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -99,7 +99,7 @@
INVALID
}
- private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
+ private static List<FunctionIdentifier> funcIdents = new ArrayList<>();
static {
funcIdents.add(AsterixBuiltinFunctions.STRING_CONTAINS);
@@ -110,7 +110,7 @@
// These function identifiers are matched in this AM's analyzeFuncExprArgs(),
// and are not visible to the outside driver.
- private static HashSet<FunctionIdentifier> secondLevelFuncIdents = new HashSet<FunctionIdentifier>();
+ private static HashSet<FunctionIdentifier> secondLevelFuncIdents = new HashSet<>();
static {
secondLevelFuncIdents.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK);
@@ -276,11 +276,12 @@
ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
// Determine whether one arg is constant, and the other is non-constant.
- ILogicalExpression constArg = null;
- ILogicalExpression nonConstArg = null;
+ ILogicalExpression constArg;
+ ILogicalExpression nonConstArg;
if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
&& arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- // The arguments of edit-distance-contains() function are asymmetrical, we can only use index if it is on the first argument
+ // The arguments of edit-distance-contains() function are asymmetrical, we can only use index if it is on
+ // the first argument
if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CONTAINS) {
return false;
}
@@ -368,11 +369,11 @@
IOptimizationContext context) throws AlgebricksException {
IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
- Dataset dataset = indexSubTree.dataset;
- ARecordType recordType = indexSubTree.recordType;
- ARecordType metaRecordType = indexSubTree.metaRecordType;
+ Dataset dataset = indexSubTree.getDataset();
+ ARecordType recordType = indexSubTree.getRecordType();
+ ARecordType metaRecordType = indexSubTree.getMetaRecordType();
// we made sure indexSubTree has datasource scan
- DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) indexSubTree.dataSourceRef.getValue();
+ DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) indexSubTree.getDataSourceRef().getValue();
InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(),
chosenIndex.getIndexType(), dataset.getDataverseName(), dataset.getDatasetName(), retainInput,
@@ -403,7 +404,7 @@
// We are optimizing a join. Add the input variable to the secondaryIndexFuncArgs.
LogicalVariable inputSearchVariable = getInputSearchVar(optFuncExpr, indexSubTree);
keyVarList.add(inputSearchVariable);
- inputOp = (AbstractLogicalOperator) probeSubTree.root;
+ inputOp = (AbstractLogicalOperator) probeSubTree.getRoot();
}
jobGenParams.setKeyVarList(keyVarList);
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
@@ -440,7 +441,7 @@
ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(null, subTree, null, chosenIndex, analysisCtx,
false, false, false, context);
// Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
- subTree.dataSourceRef.setValue(indexPlanRootOp);
+ subTree.getDataSourceRef().setValue(indexPlanRootOp);
return true;
}
@@ -451,14 +452,14 @@
boolean hasGroupBy) throws AlgebricksException {
// Figure out if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
- OptimizableOperatorSubTree indexSubTree = null;
- OptimizableOperatorSubTree probeSubTree = null;
+ OptimizableOperatorSubTree indexSubTree;
+ OptimizableOperatorSubTree probeSubTree;
// We assume that the left subtree is the outer branch and the right subtree is the inner branch.
// This assumption holds true since we only use an index from the right subtree.
// The following is just a sanity check.
if (rightSubTree.hasDataSourceScan()
- && dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+ && dataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
indexSubTree = rightSubTree;
probeSubTree = leftSubTree;
} else {
@@ -469,8 +470,8 @@
// The arguments of edit-distance-contains() function are asymmetrical, we can only use index
// if the dataset of index subtree and the dataset of first argument's subtree is the same
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CONTAINS
- && optFuncExpr.getOperatorSubTree(0).dataset != null && !optFuncExpr.getOperatorSubTree(0).dataset
- .getDatasetName().equals(indexSubTree.dataset.getDatasetName())) {
+ && optFuncExpr.getOperatorSubTree(0).getDataset() != null && !optFuncExpr.getOperatorSubTree(0)
+ .getDataset().getDatasetName().equals(indexSubTree.getDataset().getDatasetName())) {
return false;
}
@@ -489,9 +490,9 @@
// Remember the original probe subtree, and its primary-key variables,
// so we can later retrieve the missing attributes via an equi join.
- List<LogicalVariable> originalSubTreePKs = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> originalSubTreePKs = new ArrayList<>();
// Remember the primary-keys of the new probe subtree for the top-level equi join.
- List<LogicalVariable> surrogateSubTreePKs = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> surrogateSubTreePKs = new ArrayList<>();
// Copy probe subtree, replacing their variables with new ones. We will use the original variables
// to stitch together a top-level equi join.
@@ -499,8 +500,8 @@
join.getCondition().getValue(), optFuncExpr, originalSubTreePKs, surrogateSubTreePKs, context);
// Remember original live variables from the index sub tree.
- List<LogicalVariable> indexSubTreeLiveVars = new ArrayList<LogicalVariable>();
- VariableUtilities.getLiveVariables(indexSubTree.root, indexSubTreeLiveVars);
+ List<LogicalVariable> indexSubTreeLiveVars = new ArrayList<>();
+ VariableUtilities.getLiveVariables(indexSubTree.getRoot(), indexSubTreeLiveVars);
// Clone the original join condition because we may have to modify it (and we also need the original).
ILogicalExpression joinCond = join.getCondition().getValue().cloneExpression();
@@ -510,22 +511,22 @@
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK
|| optFuncExpr.getFuncExpr()
.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CONTAINS) {
- panicJoinRef = new MutableObject<ILogicalOperator>(joinRef.getValue());
- panicVarMap = new HashMap<LogicalVariable, LogicalVariable>();
+ panicJoinRef = new MutableObject<>(joinRef.getValue());
+ panicVarMap = new HashMap<>();
Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree,
probeSubTree, optFuncExpr, chosenIndex, panicVarMap, context);
- probeSubTree.rootRef.setValue(newProbeRootRef.getValue());
- probeSubTree.root = newProbeRootRef.getValue();
+ probeSubTree.getRootRef().setValue(newProbeRootRef.getValue());
+ probeSubTree.setRoot(newProbeRootRef.getValue());
}
// Create regular indexed-nested loop join path.
ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(null, indexSubTree, probeSubTree, chosenIndex,
analysisCtx, true, isLeftOuterJoin, true, context);
- indexSubTree.dataSourceRef.setValue(indexPlanRootOp);
+ indexSubTree.getDataSourceRef().setValue(indexPlanRootOp);
// Change join into a select with the same condition.
SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond), isLeftOuterJoin,
newNullPlaceHolderVar);
- topSelect.getInputs().add(indexSubTree.rootRef);
+ topSelect.getInputs().add(indexSubTree.getRootRef());
topSelect.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(topSelect);
ILogicalOperator topOp = topSelect;
@@ -535,10 +536,10 @@
LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
indexSubTreeLiveVars.addAll(originalSubTreePKs);
indexSubTreeLiveVars.add(inputSearchVar);
- List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> panicPlanLiveVars = new ArrayList<>();
VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
// Create variable mapping for union all operator.
- List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<>();
for (int i = 0; i < indexSubTreeLiveVars.size(); i++) {
LogicalVariable indexSubTreeVar = indexSubTreeLiveVars.get(i);
LogicalVariable panicPlanVar = panicVarMap.get(indexSubTreeVar);
@@ -593,7 +594,7 @@
Map<LogicalVariable, LogicalVariable> joinInputSubTreeVarMap = new HashMap<LogicalVariable, LogicalVariable>();
// Init with all live vars.
List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
- VariableUtilities.getLiveVariables(probeSubTree.root, liveVars);
+ VariableUtilities.getLiveVariables(probeSubTree.getRoot(), liveVars);
for (LogicalVariable var : liveVars) {
joinInputSubTreeVarMap.put(var, var);
}
@@ -612,25 +613,25 @@
// Create first copy.
LogicalOperatorDeepCopyWithNewVariablesVisitor firstDeepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
context, context, newProbeSubTreeVarMap);
- ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root);
+ ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.getRoot());
inferTypes(newProbeSubTree, context);
Mutable<ILogicalOperator> newProbeSubTreeRootRef = new MutableObject<ILogicalOperator>(newProbeSubTree);
// Create second copy.
LogicalOperatorDeepCopyWithNewVariablesVisitor secondDeepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
context, context, joinInputSubTreeVarMap);
- ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root);
+ ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.getRoot());
inferTypes(joinInputSubTree, context);
- probeSubTree.rootRef.setValue(joinInputSubTree);
+ probeSubTree.getRootRef().setValue(joinInputSubTree);
// Remember the original probe subtree reference so we can return it.
- Mutable<ILogicalOperator> originalProbeSubTreeRootRef = probeSubTree.rootRef;
+ Mutable<ILogicalOperator> originalProbeSubTreeRootRef = probeSubTree.getRootRef();
// Replace the original probe subtree with its copy.
- Dataset origDataset = probeSubTree.dataset;
- ARecordType origRecordType = probeSubTree.recordType;
+ Dataset origDataset = probeSubTree.getDataset();
+ ARecordType origRecordType = probeSubTree.getRecordType();
probeSubTree.initFromSubTree(newProbeSubTreeRootRef);
- probeSubTree.dataset = origDataset;
- probeSubTree.recordType = origRecordType;
+ probeSubTree.setDataset(origDataset);
+ probeSubTree.setRecordType(origRecordType);
// Replace the variables in the join condition based on the mapping of variables
// in the new probe subtree.
@@ -673,12 +674,12 @@
// We split the plan into two "branches", and add selections on each side.
AbstractLogicalOperator replicateOp = new ReplicateOperator(2);
- replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.root));
+ replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.getRoot()));
replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
context.computeAndSetTypeEnvironmentForOperator(replicateOp);
// Create select ops for removing tuples that are filterable and not filterable, respectively.
- IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.root);
+ IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.getRoot());
IAType inputSearchVarType;
if (chosenIndex.isEnforcingKeyFileds()) {
inputSearchVarType = optFuncExpr.getFieldType(optFuncExpr.findLogicalVar(inputSearchVar));
@@ -691,12 +692,12 @@
isFilterableSelectOpRef, isNotFilterableSelectOpRef);
List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
- VariableUtilities.getLiveVariables(indexSubTree.root, originalLiveVars);
+ VariableUtilities.getLiveVariables(indexSubTree.getRoot(), originalLiveVars);
// Copy the scan subtree in indexSubTree.
LogicalOperatorDeepCopyWithNewVariablesVisitor deepCopyVisitor = new LogicalOperatorDeepCopyWithNewVariablesVisitor(
context, context);
- ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root);
+ ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.getRoot());
Map<LogicalVariable, LogicalVariable> copyVarMap = deepCopyVisitor.getInputToOutputVariableMapping();
panicVarMap.putAll(copyVarMap);
@@ -1020,7 +1021,7 @@
return null;
}
- for (AbstractLogicalOperator op : subTree.assignsAndUnnests) {
+ for (AbstractLogicalOperator op : subTree.getAssignsAndUnnests()) {
if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
continue;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index d5a6f2b..f483d70 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -21,6 +21,8 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.asterix.metadata.declared.AqlDataSource;
+import org.apache.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.utils.DatasetUtils;
@@ -38,6 +40,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
@@ -59,24 +62,24 @@
NO_DATASOURCE
}
- public ILogicalOperator root = null;
- public Mutable<ILogicalOperator> rootRef = null;
- public final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<Mutable<ILogicalOperator>>();
- public final List<AbstractLogicalOperator> assignsAndUnnests = new ArrayList<AbstractLogicalOperator>();
- public Mutable<ILogicalOperator> dataSourceRef = null;
- public DataSourceType dataSourceType = DataSourceType.NO_DATASOURCE;
+ private ILogicalOperator root = null;
+ private Mutable<ILogicalOperator> rootRef = null;
+ private final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<>();
+ private final List<AbstractLogicalOperator> assignsAndUnnests = new ArrayList<>();
+ private Mutable<ILogicalOperator> dataSourceRef = null;
+ private DataSourceType dataSourceType = DataSourceType.NO_DATASOURCE;
// Dataset and type metadata. Set in setDatasetAndTypeMetadata().
- public Dataset dataset = null;
- public ARecordType recordType = null;
- public ARecordType metaRecordType = null;
+ private Dataset dataset = null;
+ private ARecordType recordType = null;
+ private ARecordType metaRecordType = null;
// Additional datasources can exist if IntroduceJoinAccessMethodRule has been applied.
// (E.g. There are index-nested-loop-joins in the plan.)
- public List<Mutable<ILogicalOperator>> ixJoinOuterAdditionalDataSourceRefs = null;
- public List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes = null;
- public List<Dataset> ixJoinOuterAdditionalDatasets = null;
- public List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
+ private List<Mutable<ILogicalOperator>> ixJoinOuterAdditionalDataSourceRefs = null;
+ private List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes = null;
+ private List<Dataset> ixJoinOuterAdditionalDatasets = null;
+ private List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) throws AlgebricksException {
reset();
@@ -102,12 +105,12 @@
if (!OperatorPropertiesUtil.isMovable(subTreeOp)) {
return false;
} else {
- assignsAndUnnestsRefs.add(subTreeOpRef);
- assignsAndUnnests.add(subTreeOp);
+ getAssignsAndUnnestsRefs().add(subTreeOpRef);
+ getAssignsAndUnnests().add(subTreeOp);
}
subTreeOpRef = subTreeOp.getInputs().get(0);
subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
- };
+ }
} while (subTreeOp.getOperatorTag() == LogicalOperatorTag.SELECT);
// Match data source (datasource scan or primary index search).
@@ -118,12 +121,12 @@
AbstractLogicalOperator subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
if (subTreeOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
- dataSourceType = DataSourceType.DATASOURCE_SCAN;
- dataSourceRef = subTreeOpRef;
+ setDataSourceType(DataSourceType.DATASOURCE_SCAN);
+ setDataSourceRef(subTreeOpRef);
return true;
} else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
- dataSourceType = DataSourceType.COLLECTION_SCAN;
- dataSourceRef = subTreeOpRef;
+ setDataSourceType(DataSourceType.COLLECTION_SCAN);
+ setDataSourceRef(subTreeOpRef);
return true;
} else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
// There can be multiple unnest-map or datasource-scan operators
@@ -141,40 +144,40 @@
AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
jobGenParams.readFromFuncArgs(f.getArguments());
if (jobGenParams.isPrimaryIndex()) {
- if (dataSourceRef == null) {
- dataSourceRef = subTreeOpRef;
- dataSourceType = DataSourceType.PRIMARY_INDEX_LOOKUP;
+ if (getDataSourceRef() == null) {
+ setDataSourceRef(subTreeOpRef);
+ setDataSourceType(DataSourceType.PRIMARY_INDEX_LOOKUP);
} else {
// One datasource already exists. This is an additional datasource.
initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
- ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.PRIMARY_INDEX_LOOKUP);
- ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+ getIxJoinOuterAdditionalDataSourceTypes().add(DataSourceType.PRIMARY_INDEX_LOOKUP);
+ getIxJoinOuterAdditionalDataSourceRefs().add(subTreeOpRef);
}
dataSourceFound = true;
}
} else if (f.getFunctionIdentifier().equals(AsterixBuiltinFunctions.EXTERNAL_LOOKUP)) {
// External lookup case
- if (dataSourceRef == null) {
- dataSourceRef = subTreeOpRef;
- dataSourceType = DataSourceType.EXTERNAL_SCAN;
+ if (getDataSourceRef() == null) {
+ setDataSourceRef(subTreeOpRef);
+ setDataSourceType(DataSourceType.EXTERNAL_SCAN);
} else {
// One datasource already exists. This is an additional datasource.
initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
- ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.EXTERNAL_SCAN);
- ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+ getIxJoinOuterAdditionalDataSourceTypes().add(DataSourceType.EXTERNAL_SCAN);
+ getIxJoinOuterAdditionalDataSourceRefs().add(subTreeOpRef);
}
dataSourceFound = true;
}
}
} else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
- ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.DATASOURCE_SCAN);
- ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+ getIxJoinOuterAdditionalDataSourceTypes().add(DataSourceType.DATASOURCE_SCAN);
+ getIxJoinOuterAdditionalDataSourceRefs().add(subTreeOpRef);
dataSourceFound = true;
} else if (subTreeOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
initializeIxJoinOuterAddtionalDataSourcesIfEmpty();
- ixJoinOuterAdditionalDataSourceTypes.add(DataSourceType.COLLECTION_SCAN);
- ixJoinOuterAdditionalDataSourceRefs.add(subTreeOpRef);
+ getIxJoinOuterAdditionalDataSourceTypes().add(DataSourceType.COLLECTION_SCAN);
+ getIxJoinOuterAdditionalDataSourceRefs().add(subTreeOpRef);
}
// Traverse the subtree while there are operators in the path.
@@ -208,14 +211,14 @@
List<Mutable<ILogicalOperator>> sourceOpRefs = new ArrayList<Mutable<ILogicalOperator>>();
List<DataSourceType> dsTypes = new ArrayList<DataSourceType>();
- sourceOpRefs.add(dataSourceRef);
- dsTypes.add(dataSourceType);
+ sourceOpRefs.add(getDataSourceRef());
+ dsTypes.add(getDataSourceType());
// If there are multiple datasources in the subtree, we need to find the dataset for these.
- if (ixJoinOuterAdditionalDataSourceRefs != null) {
- for (int i = 0; i < ixJoinOuterAdditionalDataSourceRefs.size(); i++) {
- sourceOpRefs.add(ixJoinOuterAdditionalDataSourceRefs.get(i));
- dsTypes.add(ixJoinOuterAdditionalDataSourceTypes.get(i));
+ if (getIxJoinOuterAdditionalDataSourceRefs() != null) {
+ for (int i = 0; i < getIxJoinOuterAdditionalDataSourceRefs().size(); i++) {
+ sourceOpRefs.add(getIxJoinOuterAdditionalDataSourceRefs().get(i));
+ dsTypes.add(getIxJoinOuterAdditionalDataSourceTypes().get(i));
}
}
@@ -223,6 +226,14 @@
switch (dsTypes.get(i)) {
case DATASOURCE_SCAN:
DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) sourceOpRefs.get(i).getValue();
+ IDataSource<?> datasource = dataSourceScan.getDataSource();
+ if (datasource instanceof AqlDataSource) {
+ AqlDataSourceType dsType = ((AqlDataSource) datasource).getDatasourceType();
+ if (dsType != AqlDataSourceType.INTERNAL_DATASET
+ && dsType != AqlDataSourceType.EXTERNAL_DATASET) {
+ return false;
+ }
+ }
Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
dataverseName = datasetInfo.first;
datasetName = datasetInfo.second;
@@ -244,8 +255,8 @@
break;
case COLLECTION_SCAN:
if (i != 0) {
- ixJoinOuterAdditionalDatasets.add(null);
- ixJoinOuterAdditionalRecordTypes.add(null);
+ getIxJoinOuterAdditionalDatasets().add(null);
+ getIxJoinOuterAdditionalRecordTypes().add(null);
}
continue;
case NO_DATASOURCE:
@@ -266,24 +277,24 @@
if (i == 0) {
return false;
} else {
- ixJoinOuterAdditionalDatasets.add(null);
- ixJoinOuterAdditionalRecordTypes.add(null);
+ getIxJoinOuterAdditionalDatasets().add(null);
+ getIxJoinOuterAdditionalRecordTypes().add(null);
}
}
rType = (ARecordType) itemType;
// Get the meta record type for that dataset.
- IAType metaItemType = metadataProvider.findType(ds.getMetaItemTypeDataverseName(),
- ds.getMetaItemTypeName());
+ IAType metaItemType =
+ metadataProvider.findType(ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName());
// First index is always the primary datasource in this subtree.
if (i == 0) {
- dataset = ds;
- recordType = rType;
- metaRecordType = (ARecordType) metaItemType;
+ setDataset(ds);
+ setRecordType(rType);
+ setMetaRecordType((ARecordType) metaItemType);
} else {
- ixJoinOuterAdditionalDatasets.add(ds);
- ixJoinOuterAdditionalRecordTypes.add(rType);
+ getIxJoinOuterAdditionalDatasets().add(ds);
+ getIxJoinOuterAdditionalRecordTypes().add(rType);
}
dataverseName = null;
@@ -296,14 +307,14 @@
}
public boolean hasDataSource() {
- return dataSourceType != DataSourceType.NO_DATASOURCE;
+ return getDataSourceType() != DataSourceType.NO_DATASOURCE;
}
public boolean hasIxJoinOuterAdditionalDataSource() {
boolean dataSourceFound = false;
- if (ixJoinOuterAdditionalDataSourceTypes != null) {
- for (int i = 0; i < ixJoinOuterAdditionalDataSourceTypes.size(); i++) {
- if (ixJoinOuterAdditionalDataSourceTypes.get(i) != DataSourceType.NO_DATASOURCE) {
+ if (getIxJoinOuterAdditionalDataSourceTypes() != null) {
+ for (int i = 0; i < getIxJoinOuterAdditionalDataSourceTypes().size(); i++) {
+ if (getIxJoinOuterAdditionalDataSourceTypes().get(i) != DataSourceType.NO_DATASOURCE) {
dataSourceFound = true;
break;
}
@@ -313,13 +324,13 @@
}
public boolean hasDataSourceScan() {
- return dataSourceType == DataSourceType.DATASOURCE_SCAN;
+ return getDataSourceType() == DataSourceType.DATASOURCE_SCAN;
}
public boolean hasIxJoinOuterAdditionalDataSourceScan() {
- if (ixJoinOuterAdditionalDataSourceTypes != null) {
- for (int i = 0; i < ixJoinOuterAdditionalDataSourceTypes.size(); i++) {
- if (ixJoinOuterAdditionalDataSourceTypes.get(i) == DataSourceType.DATASOURCE_SCAN) {
+ if (getIxJoinOuterAdditionalDataSourceTypes() != null) {
+ for (int i = 0; i < getIxJoinOuterAdditionalDataSourceTypes().size(); i++) {
+ if (getIxJoinOuterAdditionalDataSourceTypes().get(i) == DataSourceType.DATASOURCE_SCAN) {
return true;
}
}
@@ -328,33 +339,33 @@
}
public void reset() {
- root = null;
- rootRef = null;
- assignsAndUnnestsRefs.clear();
- assignsAndUnnests.clear();
- dataSourceRef = null;
- dataSourceType = DataSourceType.NO_DATASOURCE;
- ixJoinOuterAdditionalDataSourceRefs = null;
- ixJoinOuterAdditionalDataSourceTypes = null;
- dataset = null;
- ixJoinOuterAdditionalDatasets = null;
- recordType = null;
- ixJoinOuterAdditionalRecordTypes = null;
+ setRoot(null);
+ setRootRef(null);
+ getAssignsAndUnnestsRefs().clear();
+ getAssignsAndUnnests().clear();
+ setDataSourceRef(null);
+ setDataSourceType(DataSourceType.NO_DATASOURCE);
+ setIxJoinOuterAdditionalDataSourceRefs(null);
+ setIxJoinOuterAdditionalDataSourceTypes(null);
+ setDataset(null);
+ setIxJoinOuterAdditionalDatasets(null);
+ setRecordType(null);
+ setIxJoinOuterAdditionalRecordTypes(null);
}
public void getPrimaryKeyVars(List<LogicalVariable> target) throws AlgebricksException {
- switch (dataSourceType) {
+ switch (getDataSourceType()) {
case DATASOURCE_SCAN:
- DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) dataSourceRef.getValue();
- int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) getDataSourceRef().getValue();
+ int numPrimaryKeys = DatasetUtils.getPartitioningKeys(getDataset()).size();
for (int i = 0; i < numPrimaryKeys; i++) {
target.add(dataSourceScan.getVariables().get(i));
}
break;
case PRIMARY_INDEX_LOOKUP:
- UnnestMapOperator unnestMapOp = (UnnestMapOperator) dataSourceRef.getValue();
+ UnnestMapOperator unnestMapOp = (UnnestMapOperator) getDataSourceRef().getValue();
List<LogicalVariable> primaryKeys = null;
- primaryKeys = AccessMethodUtils.getPrimaryKeyVarsFromPrimaryUnnestMap(dataset, unnestMapOp);
+ primaryKeys = AccessMethodUtils.getPrimaryKeyVarsFromPrimaryUnnestMap(getDataset(), unnestMapOp);
target.addAll(primaryKeys);
break;
case NO_DATASOURCE:
@@ -364,14 +375,14 @@
}
public List<LogicalVariable> getDataSourceVariables() throws AlgebricksException {
- switch (dataSourceType) {
+ switch (getDataSourceType()) {
case DATASOURCE_SCAN:
case EXTERNAL_SCAN:
case PRIMARY_INDEX_LOOKUP:
- AbstractScanOperator scanOp = (AbstractScanOperator) dataSourceRef.getValue();
+ AbstractScanOperator scanOp = (AbstractScanOperator) getDataSourceRef().getValue();
return scanOp.getVariables();
case COLLECTION_SCAN:
- return new ArrayList<LogicalVariable>();
+ return new ArrayList<>();
case NO_DATASOURCE:
default:
throw new AlgebricksException("The subtree does not have any data source.");
@@ -379,16 +390,16 @@
}
public List<LogicalVariable> getIxJoinOuterAdditionalDataSourceVariables(int idx) throws AlgebricksException {
- if (ixJoinOuterAdditionalDataSourceRefs != null && ixJoinOuterAdditionalDataSourceRefs.size() > idx) {
- switch (ixJoinOuterAdditionalDataSourceTypes.get(idx)) {
+ if (getIxJoinOuterAdditionalDataSourceRefs() != null && getIxJoinOuterAdditionalDataSourceRefs().size() > idx) {
+ switch (getIxJoinOuterAdditionalDataSourceTypes().get(idx)) {
case DATASOURCE_SCAN:
case EXTERNAL_SCAN:
case PRIMARY_INDEX_LOOKUP:
- AbstractScanOperator scanOp = (AbstractScanOperator) ixJoinOuterAdditionalDataSourceRefs.get(idx)
- .getValue();
+ AbstractScanOperator scanOp =
+ (AbstractScanOperator) getIxJoinOuterAdditionalDataSourceRefs().get(idx).getValue();
return scanOp.getVariables();
case COLLECTION_SCAN:
- return new ArrayList<LogicalVariable>();
+ return new ArrayList<>();
case NO_DATASOURCE:
default:
throw new AlgebricksException("The subtree does not have any additional data sources.");
@@ -399,12 +410,109 @@
}
public void initializeIxJoinOuterAddtionalDataSourcesIfEmpty() {
- if (ixJoinOuterAdditionalDataSourceRefs == null) {
- ixJoinOuterAdditionalDataSourceRefs = new ArrayList<Mutable<ILogicalOperator>>();
- ixJoinOuterAdditionalDataSourceTypes = new ArrayList<DataSourceType>();
- ixJoinOuterAdditionalDatasets = new ArrayList<Dataset>();
- ixJoinOuterAdditionalRecordTypes = new ArrayList<ARecordType>();
+ if (getIxJoinOuterAdditionalDataSourceRefs() == null) {
+ setIxJoinOuterAdditionalDataSourceRefs(new ArrayList<Mutable<ILogicalOperator>>());
+ setIxJoinOuterAdditionalDataSourceTypes(new ArrayList<DataSourceType>());
+ setIxJoinOuterAdditionalDatasets(new ArrayList<Dataset>());
+ setIxJoinOuterAdditionalRecordTypes(new ArrayList<ARecordType>());
}
}
+ public ILogicalOperator getRoot() {
+ return root;
+ }
+
+ public void setRoot(ILogicalOperator root) {
+ this.root = root;
+ }
+
+ public Mutable<ILogicalOperator> getRootRef() {
+ return rootRef;
+ }
+
+ public void setRootRef(Mutable<ILogicalOperator> rootRef) {
+ this.rootRef = rootRef;
+ }
+
+ public List<Mutable<ILogicalOperator>> getAssignsAndUnnestsRefs() {
+ return assignsAndUnnestsRefs;
+ }
+
+ public List<AbstractLogicalOperator> getAssignsAndUnnests() {
+ return assignsAndUnnests;
+ }
+
+ public Mutable<ILogicalOperator> getDataSourceRef() {
+ return dataSourceRef;
+ }
+
+ public void setDataSourceRef(Mutable<ILogicalOperator> dataSourceRef) {
+ this.dataSourceRef = dataSourceRef;
+ }
+
+ public DataSourceType getDataSourceType() {
+ return dataSourceType;
+ }
+
+ public void setDataSourceType(DataSourceType dataSourceType) {
+ this.dataSourceType = dataSourceType;
+ }
+
+ public Dataset getDataset() {
+ return dataset;
+ }
+
+ public void setDataset(Dataset dataset) {
+ this.dataset = dataset;
+ }
+
+ public ARecordType getRecordType() {
+ return recordType;
+ }
+
+ public void setRecordType(ARecordType recordType) {
+ this.recordType = recordType;
+ }
+
+ public ARecordType getMetaRecordType() {
+ return metaRecordType;
+ }
+
+ public void setMetaRecordType(ARecordType metaRecordType) {
+ this.metaRecordType = metaRecordType;
+ }
+
+ public List<Mutable<ILogicalOperator>> getIxJoinOuterAdditionalDataSourceRefs() {
+ return ixJoinOuterAdditionalDataSourceRefs;
+ }
+
+ public void setIxJoinOuterAdditionalDataSourceRefs(
+ List<Mutable<ILogicalOperator>> ixJoinOuterAdditionalDataSourceRefs) {
+ this.ixJoinOuterAdditionalDataSourceRefs = ixJoinOuterAdditionalDataSourceRefs;
+ }
+
+ public List<DataSourceType> getIxJoinOuterAdditionalDataSourceTypes() {
+ return ixJoinOuterAdditionalDataSourceTypes;
+ }
+
+ public void setIxJoinOuterAdditionalDataSourceTypes(List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes) {
+ this.ixJoinOuterAdditionalDataSourceTypes = ixJoinOuterAdditionalDataSourceTypes;
+ }
+
+ public List<Dataset> getIxJoinOuterAdditionalDatasets() {
+ return ixJoinOuterAdditionalDatasets;
+ }
+
+ public void setIxJoinOuterAdditionalDatasets(List<Dataset> ixJoinOuterAdditionalDatasets) {
+ this.ixJoinOuterAdditionalDatasets = ixJoinOuterAdditionalDatasets;
+ }
+
+ public List<ARecordType> getIxJoinOuterAdditionalRecordTypes() {
+ return ixJoinOuterAdditionalRecordTypes;
+ }
+
+ public void setIxJoinOuterAdditionalRecordTypes(List<ARecordType> ixJoinOuterAdditionalRecordTypes) {
+ this.ixJoinOuterAdditionalRecordTypes = ixJoinOuterAdditionalRecordTypes;
+ }
+
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index eba2d33..c3c162e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -50,10 +50,8 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
/**
@@ -61,13 +59,13 @@
*/
public class RTreeAccessMethod implements IAccessMethod {
- private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
+ private static List<FunctionIdentifier> funcIdents = new ArrayList<>();
static {
funcIdents.add(AsterixBuiltinFunctions.SPATIAL_INTERSECT);
}
- public static RTreeAccessMethod INSTANCE = new RTreeAccessMethod();
+ public static final RTreeAccessMethod INSTANCE = new RTreeAccessMethod();
@Override
public List<FunctionIdentifier> getOptimizableFunctions() {
@@ -107,7 +105,7 @@
return false;
}
// Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
- subTree.dataSourceRef.setValue(primaryIndexUnnestOp);
+ subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
return true;
}
@@ -125,16 +123,17 @@
OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean isLeftOuterJoin,
boolean hasGroupBy) throws AlgebricksException {
- // Determine if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
+ // Determine if the index is applicable on the left or right side (if both, we arbitrarily prefer the left
+ // side).
Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
- OptimizableOperatorSubTree indexSubTree = null;
- OptimizableOperatorSubTree probeSubTree = null;
+ OptimizableOperatorSubTree indexSubTree;
+ OptimizableOperatorSubTree probeSubTree;
// We assume that the left subtree is the outer branch and the right subtree is the inner branch.
// This assumption holds true since we only use an index from the right subtree.
// The following is just a sanity check.
if (rightSubTree.hasDataSourceScan()
- && dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+ && dataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
indexSubTree = rightSubTree;
probeSubTree = leftSubTree;
} else {
@@ -143,8 +142,8 @@
LogicalVariable newNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
- //get a new null place holder variable that is the first field variable of the primary key
- //from the indexSubTree's datasourceScanOp
+ // get a new null place holder variable that is the first field variable of the primary key
+ // from the indexSubTree's datasourceScanOp
newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
}
@@ -156,15 +155,15 @@
}
if (isLeftOuterJoin && hasGroupBy) {
- //reset the null place holder variable
+ // reset the null place holder variable
AccessMethodUtils.resetLOJNullPlaceholderVariableInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
}
- indexSubTree.dataSourceRef.setValue(primaryIndexUnnestOp);
+ indexSubTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
// Change join into a select with the same condition.
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
SelectOperator topSelect = new SelectOperator(joinOp.getCondition(), isLeftOuterJoin, newNullPlaceHolderVar);
- topSelect.getInputs().add(indexSubTree.rootRef);
+ topSelect.getInputs().add(indexSubTree.getRootRef());
topSelect.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(topSelect);
// Replace the original join with the new subtree rooted at the select op.
@@ -175,12 +174,12 @@
private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context)
- throws AlgebricksException {
+ throws AlgebricksException {
IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
- Dataset dataset = indexSubTree.dataset;
- ARecordType recordType = indexSubTree.recordType;
- ARecordType metaRecordType = indexSubTree.metaRecordType;
+ Dataset dataset = indexSubTree.getDataset();
+ ARecordType recordType = indexSubTree.getRecordType();
+ ARecordType metaRecordType = indexSubTree.getMetaRecordType();
int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(chosenIndex, analysisCtx);
Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
@@ -194,16 +193,18 @@
int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numSecondaryKeys = numDimensions * 2;
// we made sure indexSubTree has datasource scan
- AbstractDataSourceOperator dataSourceOp = (AbstractDataSourceOperator) indexSubTree.dataSourceRef.getValue();
+ AbstractDataSourceOperator dataSourceOp = (AbstractDataSourceOperator) indexSubTree.getDataSourceRef()
+ .getValue();
RTreeJobGenParams jobGenParams = new RTreeJobGenParams(chosenIndex.getIndexName(), IndexType.RTREE,
dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
// A spatial object is serialized in the constant of the func expr we are optimizing.
// The R-Tree expects as input an MBR represented with 1 field per dimension.
- // Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple (as the R-Tree expects them).
+ // Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple (as the
+ // R-Tree expects them).
// List of variables for the assign.
- ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> keyVarList = new ArrayList<>();
// List of expressions for the assign.
- ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+ ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<>();
Pair<ILogicalExpression, Boolean> returnedSearchKeyExpr = AccessMethodUtils.createSearchKeyExpr(optFuncExpr,
indexSubTree, probeSubTree);
ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
@@ -237,23 +238,18 @@
assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
} else {
// We are optimizing a join, place the assign op top of the probe subtree.
- assignSearchKeys.getInputs().add(probeSubTree.rootRef);
+ assignSearchKeys.getInputs().add(probeSubTree.getRootRef());
}
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, retainInput, retainNull);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- UnnestMapOperator externalDataAccessOp = AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp,
- dataset, recordType, secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull);
- return externalDataAccessOp;
- } else {
- AbstractUnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceOp,
- dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true, retainInput, false,
- false);
- return primaryIndexUnnestOp;
- }
+ return dataset.getDatasetType() == DatasetType.EXTERNAL
+ ? AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp, dataset, recordType,
+ secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull)
+ : AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceOp, dataset, recordType, metaRecordType,
+ secondaryIndexUnnestOp, context, true, retainInput, false, false);
}
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index 9030da9..bf7b975 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -24,18 +24,18 @@
import java.util.logging.Logger;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.InsertStatement;
import org.apache.asterix.lang.common.statement.NodeGroupDropStatement;
import org.apache.asterix.metadata.dataset.hints.DatasetHints;
import org.apache.asterix.metadata.entities.AsterixBuiltinTypeMap;
import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.util.AsterixAppContextInfo;
import org.apache.asterix.om.util.AsterixClusterProperties;
@@ -149,7 +149,7 @@
break;
case Statement.Kind.DATASET_DROP:
- DropStatement dropStmt = (DropStatement) stmt;
+ DropDatasetStatement dropStmt = (DropDatasetStatement) stmt;
if (dropStmt.getDataverseName() != null) {
dataverse = dropStmt.getDataverseName().getValue();
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
index 79342c9..5c09978 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -30,8 +30,8 @@
import org.apache.asterix.lang.aql.visitor.base.IAQLVisitor;
import org.apache.asterix.lang.common.base.Clause;
import org.apache.asterix.lang.common.base.Expression;
-import org.apache.asterix.lang.common.base.ILangExpression;
import org.apache.asterix.lang.common.base.Expression.Kind;
+import org.apache.asterix.lang.common.base.ILangExpression;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 334436a..05bd343 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -81,7 +81,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDecl;
@@ -1218,7 +1218,7 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visit(DropStatement del, Mutable<ILogicalOperator> arg)
+ public Pair<ILogicalOperator, LogicalVariable> visit(DropDatasetStatement del, Mutable<ILogicalOperator> arg)
throws AsterixException {
// TODO Auto-generated method stub
return null;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
new file mode 100644
index 0000000..3c24cbc
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -0,0 +1,129 @@
+/*
+ * 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.translator;
+
+import java.rmi.RemoteException;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.metadata.declared.AqlMetadataProvider;
+import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.json.JSONException;
+
+/**
+ * An interface that takes care of executing a list of statements that are submitted through an Asterix API
+ */
+public interface IStatementExecutor {
+
+ /**
+ * Specifies result delivery of executed statements
+ */
+ public enum ResultDelivery {
+ /**
+ * Wait for results to be read
+ */
+ SYNC,
+ /**
+ * Flush out result handle beofre waiting for the result
+ */
+ ASYNC,
+ /**
+ * Return result handle and don't wait for the result
+ */
+ ASYNC_DEFERRED
+ }
+
+ public static class Stats {
+ private long count;
+ private long size;
+
+ public long getCount() {
+ return count;
+ }
+
+ public void setCount(long count) {
+ this.count = count;
+ }
+
+ public long getSize() {
+ return size;
+ }
+
+ public void setSize(long size) {
+ this.size = size;
+ }
+
+ }
+
+ /**
+ * Compiles and execute a list of statements.
+ *
+ * @param hcc
+ * A Hyracks client connection that is used to submit a jobspec to Hyracks.
+ * @param hdc
+ * A Hyracks dataset client object that is used to read the results.
+ * @param resultDelivery
+ * The {@code ResultDelivery} kind required for queries in the list of statements
+ * @throws Exception
+ */
+ void compileAndExecute(IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery)
+ throws Exception;
+
+ /**
+ * Compiles and execute a list of statements.
+ *
+ * @param hcc
+ * A Hyracks client connection that is used to submit a jobspec to Hyracks.
+ * @param hdc
+ * A Hyracks dataset client object that is used to read the results.
+ * @param resultDelivery
+ * The {@code ResultDelivery} kind required for queries in the list of statements
+ * @param stats
+ * a reference to write the stats of executed queries
+ * @throws Exception
+ */
+ void compileAndExecute(IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery,
+ Stats stats) throws Exception;
+
+ /**
+ * rewrites and compiles query into a hyracks job specifications
+ *
+ * @param metadataProvider
+ * The metadataProvider used to access metadata and build runtimes
+ * @param query
+ * The query to be compiled
+ * @param dmlStatement
+ * The data modification statement when the query results in a modification to a dataset
+ * @return the compiled {@code JobSpecification}
+ * @throws AsterixException
+ * @throws RemoteException
+ * @throws AlgebricksException
+ * @throws JSONException
+ * @throws ACIDException
+ */
+ JobSpecification rewriteCompileQuery(AqlMetadataProvider metadataProvider, Query query,
+ ICompiledDmlStatement dmlStatement)
+ throws AsterixException, RemoteException, AlgebricksException, JSONException, ACIDException;
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
new file mode 100644
index 0000000..9addc87
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
@@ -0,0 +1,46 @@
+/*
+ * 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.translator;
+
+import java.util.List;
+
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.Statement;
+
+/**
+ * An interface for creating a statement executor which can be provided by
+ * implementing {@code IStatementExecutorExtension}
+ */
+@FunctionalInterface
+public interface IStatementExecutorFactory {
+
+ /**
+ * create a statement executor
+ *
+ * @param statements
+ * Statements to execute
+ * @param conf
+ * request configuration
+ * @param compilationProvider
+ * @return an implementation of {@code IStatementExecutor} that is used to execute the passed list of statements
+ */
+ IStatementExecutor create(List<Statement> statements, SessionConfig conf,
+ ILangCompilationProvider compilationProvider);
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index bd87e6e..ed9c1e6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -18,8 +18,6 @@
*/
package org.apache.asterix.translator;
-import static java.util.logging.Logger.global;
-
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
@@ -29,8 +27,8 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import java.util.Map.Entry;
+import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
@@ -41,15 +39,15 @@
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.aql.util.RangeMapBuilder;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.base.ILangExpression;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
import org.apache.asterix.lang.common.clause.OrderbyClause;
-import org.apache.asterix.lang.common.clause.WhereClause;
import org.apache.asterix.lang.common.clause.OrderbyClause.OrderModifier;
+import org.apache.asterix.lang.common.clause.WhereClause;
import org.apache.asterix.lang.common.expression.CallExpr;
import org.apache.asterix.lang.common.expression.FieldAccessor;
import org.apache.asterix.lang.common.expression.FieldBinding;
@@ -57,14 +55,14 @@
import org.apache.asterix.lang.common.expression.IfExpr;
import org.apache.asterix.lang.common.expression.IndexAccessor;
import org.apache.asterix.lang.common.expression.ListConstructor;
+import org.apache.asterix.lang.common.expression.ListConstructor.Type;
import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.OperatorExpr;
import org.apache.asterix.lang.common.expression.QuantifiedExpression;
+import org.apache.asterix.lang.common.expression.QuantifiedExpression.Quantifier;
import org.apache.asterix.lang.common.expression.RecordConstructor;
import org.apache.asterix.lang.common.expression.UnaryExpr;
import org.apache.asterix.lang.common.expression.VariableExpr;
-import org.apache.asterix.lang.common.expression.ListConstructor.Type;
-import org.apache.asterix.lang.common.expression.QuantifiedExpression.Quantifier;
import org.apache.asterix.lang.common.literal.StringLiteral;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
@@ -75,13 +73,13 @@
import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.metadata.declared.AqlSourceId;
import org.apache.asterix.metadata.declared.DatasetDataSource;
import org.apache.asterix.metadata.declared.LoadableDataSource;
import org.apache.asterix.metadata.declared.ResultSetDataSink;
import org.apache.asterix.metadata.declared.ResultSetSinkId;
-import org.apache.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -117,15 +115,15 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation.BroadcastSide;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
-import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation.BroadcastSide;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
@@ -141,13 +139,13 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalOperatorDeepCopyWithNewVariablesVisitor;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
@@ -198,10 +196,10 @@
"Unable to load dataset " + clffs.getDatasetName() + " since it does not exist");
}
IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
- IAType metaItemType =
- metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
- DatasetDataSource targetDatasource =
- validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+ IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+ dataset.getMetaItemTypeName());
+ DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
+ stmt.getDatasetName());
List<List<String>> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
if (dataset.hasMetaPart()) {
throw new AlgebricksException(
@@ -264,8 +262,8 @@
additionalFilteringExpressions = new ArrayList<>();
PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, payloadVar, additionalFilteringVars,
additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
- additionalFilteringAssign =
- new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
+ additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
+ additionalFilteringAssignExpressions);
}
InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, payloadRef,
@@ -284,12 +282,11 @@
return new ALogicalPlanImpl(new MutableObject<>(leafOperator));
}
- @SuppressWarnings("unchecked")
@Override
public ILogicalPlan translate(Query expr, String outputDatasetName, ICompiledDmlStatement stmt)
throws AlgebricksException {
- Pair<ILogicalOperator, LogicalVariable> p =
- expr.accept(this, new MutableObject<>(new EmptyTupleSourceOperator()));
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+ new MutableObject<>(new EmptyTupleSourceOperator()));
ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<>();
ILogicalOperator topOp = p.first;
ProjectOperator project = (ProjectOperator) topOp;
@@ -322,7 +319,10 @@
* because dataset only accept non-collection records
*/
LogicalVariable seqVar = context.newVar();
- /** This assign adds a marker function collection-to-sequence: if the input is a singleton collection, unnest it; otherwise do nothing. */
+ /**
+ * This assign adds a marker function collection-to-sequence: if the input is a singleton collection, unnest
+ * it; otherwise do nothing.
+ */
AssignOperator assignCollectionToSequence = new AssignOperator(seqVar,
new MutableObject<>(new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.COLLECTION_TO_SEQUENCE),
@@ -331,11 +331,10 @@
project.getInputs().get(0).setValue(assignCollectionToSequence);
project.getVariables().set(0, seqVar);
resVar = seqVar;
- DatasetDataSource targetDatasource =
- validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
- List<Integer> keySourceIndicator =
- ((InternalDatasetDetails) targetDatasource.getDataset().getDatasetDetails())
- .getKeySourceIndicator();
+ DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
+ stmt.getDatasetName());
+ List<Integer> keySourceIndicator = ((InternalDatasetDetails) targetDatasource.getDataset()
+ .getDatasetDetails()).getKeySourceIndicator();
ArrayList<LogicalVariable> vars = new ArrayList<>();
ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<>();
@@ -367,8 +366,8 @@
PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, resVar, additionalFilteringVars,
additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
- additionalFilteringAssign =
- new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
+ additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
+ additionalFilteringAssignExpressions);
additionalFilteringAssign.getInputs().add(new MutableObject<>(project));
assign.getInputs().add(new MutableObject<>(additionalFilteringAssign));
} else {
@@ -384,7 +383,8 @@
break;
case Statement.Kind.UPSERT:
leafOperator = translateUpsert(targetDatasource, varRef, varRefsForLoading,
- additionalFilteringExpressions, assign, additionalFilteringField);
+ additionalFilteringExpressions, assign, additionalFilteringField, unnestVar, project, exprs,
+ resVar, additionalFilteringAssign);
break;
case Statement.Kind.DELETE:
leafOperator = translateDelete(targetDatasource, varRef, varRefsForLoading,
@@ -451,8 +451,8 @@
List<LogicalVariable> metaAndKeysVars = null;
List<Mutable<ILogicalExpression>> metaAndKeysExprs = null;
List<Mutable<ILogicalExpression>> metaExpSingletonList = null;
- boolean isChangeFeed =
- FeedMetadataUtil.isChangeFeed(metadataProvider, sfs.getDataverseName(), sfs.getFeedName());
+ boolean isChangeFeed = FeedMetadataUtil.isChangeFeed(metadataProvider, sfs.getDataverseName(),
+ sfs.getFeedName());
if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
metaAndKeysVars = new ArrayList<>();
metaAndKeysExprs = new ArrayList<>();
@@ -527,26 +527,94 @@
private ILogicalOperator translateUpsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
List<Mutable<ILogicalExpression>> varRefsForLoading,
List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
- List<String> additionalFilteringField) throws AlgebricksException {
- if (targetDatasource.getDataset().hasMetaPart()) {
+ List<String> additionalFilteringField, LogicalVariable unnestVar, ProjectOperator project,
+ List<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar, AssignOperator additionalFilteringAssign)
+ throws AlgebricksException {
+ if (!targetDatasource.getDataset().allow(project, Dataset.OP_UPSERT)) {
throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
+ ": upsert into dataset is not supported on Datasets with Meta records");
}
- InsertDeleteUpsertOperator upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteUpsertOperator.Kind.UPSERT, false);
- upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
- upsertOp.getInputs().add(new MutableObject<>(assign));
- // Create and add a new variable used for representing the original record
- ARecordType recordType = (ARecordType) targetDatasource.getItemType();
- upsertOp.setPrevRecordVar(context.newVar());
- upsertOp.setPrevRecordType(recordType);
- if (additionalFilteringField != null) {
- upsertOp.setPrevFilterVar(context.newVar());
- upsertOp.setPrevFilterType(recordType.getFieldType(additionalFilteringField.get(0)));
+ if (targetDatasource.getDataset().hasMetaPart()) {
+ InsertDeleteUpsertOperator feedModificationOp;
+ AssignOperator metaAndKeysAssign;
+ List<LogicalVariable> metaAndKeysVars;
+ List<Mutable<ILogicalExpression>> metaAndKeysExprs;
+ List<Mutable<ILogicalExpression>> metaExpSingletonList;
+ metaAndKeysVars = new ArrayList<>();
+ metaAndKeysExprs = new ArrayList<>();
+ // add the meta function
+ IFunctionInfo finfoMeta = FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.META);
+ ScalarFunctionCallExpression metaFunction = new ScalarFunctionCallExpression(finfoMeta,
+ new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+ // create assign for the meta part
+ LogicalVariable metaVar = context.newVar();
+ metaExpSingletonList = new ArrayList<>(1);
+ metaExpSingletonList.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
+ metaAndKeysVars.add(metaVar);
+ metaAndKeysExprs.add(new MutableObject<>(metaFunction));
+ project.getVariables().add(metaVar);
+ varRefsForLoading.clear();
+ for (Mutable<ILogicalExpression> assignExpr : exprs) {
+ if (assignExpr.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression funcCall = (AbstractFunctionCallExpression) assignExpr.getValue();
+ funcCall.substituteVar(resVar, unnestVar);
+ LogicalVariable pkVar = context.newVar();
+ metaAndKeysVars.add(pkVar);
+ metaAndKeysExprs.add(new MutableObject<>(assignExpr.getValue()));
+ project.getVariables().add(pkVar);
+ varRefsForLoading.add(new MutableObject<>(new VariableReferenceExpression(pkVar)));
+ }
+ }
+ // A change feed, we don't need the assign to access PKs
+ feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
+ metaExpSingletonList, InsertDeleteUpsertOperator.Kind.UPSERT, false);
+ // Create and add a new variable used for representing the original record
+ feedModificationOp.setPrevRecordVar(context.newVar());
+ feedModificationOp.setPrevRecordType(targetDatasource.getItemType());
+ if (targetDatasource.getDataset().hasMetaPart()) {
+ List<LogicalVariable> metaVars = new ArrayList<>();
+ metaVars.add(context.newVar());
+ feedModificationOp.setPrevAdditionalNonFilteringVars(metaVars);
+ List<Object> metaTypes = new ArrayList<>();
+ metaTypes.add(targetDatasource.getMetaItemType());
+ feedModificationOp.setPrevAdditionalNonFilteringTypes(metaTypes);
+ }
+
+ if (additionalFilteringField != null) {
+ feedModificationOp.setPrevFilterVar(context.newVar());
+ feedModificationOp.setPrevFilterType(
+ ((ARecordType) targetDatasource.getItemType()).getFieldType(additionalFilteringField.get(0)));
+ additionalFilteringAssign.getInputs().clear();
+ additionalFilteringAssign.getInputs().add(assign.getInputs().get(0));
+ feedModificationOp.getInputs().add(new MutableObject<>(additionalFilteringAssign));
+ } else {
+ feedModificationOp.getInputs().add(assign.getInputs().get(0));
+ }
+ metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
+ metaAndKeysAssign.getInputs().add(project.getInputs().get(0));
+ project.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
+ feedModificationOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ SinkOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<>(feedModificationOp));
+ return leafOperator;
+ } else {
+ InsertDeleteUpsertOperator feedModificationOp;
+ feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
+ InsertDeleteUpsertOperator.Kind.UPSERT, false);
+ feedModificationOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ feedModificationOp.getInputs().add(new MutableObject<>(assign));
+ // Create and add a new variable used for representing the original record
+ ARecordType recordType = (ARecordType) targetDatasource.getItemType();
+ feedModificationOp.setPrevRecordVar(context.newVar());
+ feedModificationOp.setPrevRecordType(recordType);
+ if (additionalFilteringField != null) {
+ feedModificationOp.setPrevFilterVar(context.newVar());
+ feedModificationOp.setPrevFilterType(recordType.getFieldType(additionalFilteringField.get(0)));
+ }
+ SinkOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<>(feedModificationOp));
+ return leafOperator;
}
- SinkOperator leafOperator = new SinkOperator();
- leafOperator.getInputs().add(new MutableObject<>(upsertOp));
- return leafOperator;
}
private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
@@ -577,8 +645,8 @@
}
AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
- IAType metaItemType =
- metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
+ IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+ dataset.getMetaItemTypeName());
INodeDomain domain = metadataProvider.findNodeDomain(dataset.getNodeGroupName());
return new DatasetDataSource(sourceId, dataset, itemType, metaItemType, AqlDataSourceType.INTERNAL_DATASET,
dataset.getDatasetDetails(), domain);
@@ -586,8 +654,8 @@
private FileSplit getDefaultOutputFileLocation() throws MetadataException {
String outputDir = System.getProperty("java.io.tmpDir");
- String filePath =
- outputDir + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
+ String filePath = outputDir + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
+ + outputFileID.incrementAndGet();
AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
return new FileSplit(metadataProperties.getMetadataNodeName(), new FileReference(new File(filePath)));
}
@@ -604,8 +672,8 @@
returnedOp.getInputs().add(tupSource);
} else {
v = context.newVar(lc.getVarExpr());
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo =
- langExprToAlgExpression(lc.getBindingExpr(), tupSource);
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(lc.getBindingExpr(),
+ tupSource);
returnedOp = new AssignOperator(v, new MutableObject<>(eo.first));
returnedOp.getInputs().add(eo.second);
}
@@ -620,8 +688,8 @@
AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
fldAccess.getArguments().add(new MutableObject<>(p.first));
- ILogicalExpression faExpr =
- new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent().getValue())));
+ ILogicalExpression faExpr = new ConstantExpression(
+ new AsterixConstantValue(new AString(fa.getIdent().getValue())));
fldAccess.getArguments().add(new MutableObject<>(faExpr));
AssignOperator a = new AssignOperator(v, new MutableObject<>(fldAccess));
a.getInputs().add(p.second);
@@ -639,8 +707,8 @@
FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
f.getArguments().add(new MutableObject<>(p.first));
} else {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> indexPair =
- langExprToAlgExpression(ia.getIndexExpr(), tupSource);
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> indexPair = langExprToAlgExpression(ia.getIndexExpr(),
+ tupSource);
f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
f.getArguments().add(new MutableObject<>(p.first));
f.getArguments().add(new MutableObject<>(indexPair.first));
@@ -747,8 +815,8 @@
if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
- UnnestingFunctionCallExpression ufce =
- new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(fi), args);
+ UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(fi),
+ args);
ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
f = ufce;
} else {
@@ -762,7 +830,6 @@
throw new IllegalStateException("Function declarations should be inlined at AST rewriting phase.");
}
- @SuppressWarnings("unchecked")
@Override
public Pair<ILogicalOperator, LogicalVariable> visit(GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
throws AsterixException {
@@ -771,9 +838,8 @@
List<Pair<Expression, Identifier>> groupFieldList = gc.getGroupFieldList();
List<Mutable<ILogicalExpression>> groupRecordConstructorArgList = new ArrayList<>();
for (Pair<Expression, Identifier> groupField : groupFieldList) {
- ILogicalExpression groupFieldNameExpr =
- langExprToAlgExpression(new LiteralExpr(new StringLiteral(groupField.second.getValue())),
- topOp).first;
+ ILogicalExpression groupFieldNameExpr = langExprToAlgExpression(
+ new LiteralExpr(new StringLiteral(groupField.second.getValue())), topOp).first;
groupRecordConstructorArgList.add(new MutableObject<>(groupFieldNameExpr));
ILogicalExpression groupFieldExpr = langExprToAlgExpression(groupField.first, topOp).first;
groupRecordConstructorArgList.add(new MutableObject<>(groupFieldExpr));
@@ -802,14 +868,13 @@
// Calls the first-element aggregate function on a decoration variable to make sure the value
// is propagated through a global aggregation.
AggregateFunctionCallExpression firstElementAgg = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.FIRST_ELEMENT, flArgs);
+ .makeAggregateFunctionExpression(AsterixBuiltinFunctions.FIRST_ELEMENT, flArgs);
aggVars.add(decorVar);
aggFuncs.add(new MutableObject<>(firstElementAgg));
}
for (Entry<Expression, VariableExpr> entry : gc.getWithVarMap().entrySet()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> listifyInput =
- langExprToAlgExpression(entry.getKey(), topOp);
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> listifyInput = langExprToAlgExpression(
+ entry.getKey(), topOp);
topOp = listifyInput.second;
List<Mutable<ILogicalExpression>> flArgs = new ArrayList<>();
flArgs.add(new MutableObject<>(listifyInput.first));
@@ -892,7 +957,7 @@
Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
LogicalVariable varCond = pCond.second;
- //Creates a subplan for the "then" branch.
+ // Creates a subplan for the "then" branch.
Pair<ILogicalOperator, LogicalVariable> opAndVarForThen = constructSubplanOperatorForBranch(pCond.first,
new MutableObject<>(new VariableReferenceExpression(varCond)), ifexpr.getThenExpr());
@@ -925,8 +990,8 @@
// Produces the final result.
LogicalVariable resultVar = context.newVar();
- AssignOperator finalAssignOp =
- new AssignOperator(resultVar, new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+ AssignOperator finalAssignOp = new AssignOperator(resultVar,
+ new MutableObject<>(new VariableReferenceExpression(unnestVar)));
finalAssignOp.getInputs().add(new MutableObject<>(unnestOp));
return new Pair<>(finalAssignOp, resultVar);
}
@@ -1222,29 +1287,22 @@
return new ScalarFunctionCallExpression(finfo);
}
- private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+ private static FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
switch (t) {
- case EQ: {
+ case EQ:
return AlgebricksBuiltinFunctions.EQ;
- }
- case NEQ: {
+ case NEQ:
return AlgebricksBuiltinFunctions.NEQ;
- }
- case GT: {
+ case GT:
return AlgebricksBuiltinFunctions.GT;
- }
- case GE: {
+ case GE:
return AlgebricksBuiltinFunctions.GE;
- }
- case LT: {
+ case LT:
return AlgebricksBuiltinFunctions.LT;
- }
- case LE: {
+ case LE:
return AlgebricksBuiltinFunctions.LE;
- }
- default: {
+ default:
throw new IllegalStateException();
- }
}
}
@@ -1300,8 +1358,8 @@
Mutable<ILogicalOperator> topOpRef) throws AsterixException {
switch (expr.getKind()) {
case VARIABLE_EXPRESSION:
- VariableReferenceExpression ve =
- new VariableReferenceExpression(context.getVar(((VariableExpr) expr).getVar().getId()));
+ VariableReferenceExpression ve = new VariableReferenceExpression(
+ context.getVar(((VariableExpr) expr).getVar().getId()));
return new Pair<>(ve, topOpRef);
case LITERAL_EXPRESSION:
LiteralExpr val = (LiteralExpr) expr;
@@ -1344,7 +1402,7 @@
LogicalVariable varListified = context.newSubplanOutputVar();
AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
- (List) mkSingletonArrayList(new MutableObject<>(funAgg)));
+ mkSingletonArrayList(new MutableObject<>(funAgg)));
agg.getInputs().add(opRef);
ILogicalOperator res;
if (bProject) {
@@ -1510,8 +1568,8 @@
// Recursively eliminate shared operator reference for the operator subtree,
// even if it is a deep copy of some other one.
- Map<LogicalVariable, LogicalVariable> childVarMap =
- eliminateSharedOperatorReference(childRef, opRefSet);
+ Map<LogicalVariable, LogicalVariable> childVarMap = eliminateSharedOperatorReference(childRef,
+ opRefSet);
// Substitute variables according to the new subtree.
VariableUtilities.substituteVariables(currentOperator, childVarMap, null);
@@ -1554,8 +1612,8 @@
context.enterSubplan();
SubplanOperator subplanOp = new SubplanOperator();
subplanOp.getInputs().add(new MutableObject<>(inputOp));
- Mutable<ILogicalOperator> nestedSource =
- new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
+ Mutable<ILogicalOperator> nestedSource = new MutableObject<>(
+ new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
SelectOperator select = new SelectOperator(selectExpr, false, null);
// The select operator cannot be moved up and down, otherwise it will cause typing issues (ASTERIXDB-1203).
OperatorPropertiesUtil.markMovable(select, false);
@@ -1575,8 +1633,8 @@
// Processes EXISTS and NOT EXISTS.
private AssignOperator processExists(ILogicalExpression inputExpr, LogicalVariable v1, boolean not) {
- AbstractFunctionCallExpression count =
- new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SCALAR_COUNT));
+ AbstractFunctionCallExpression count = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SCALAR_COUNT));
count.getArguments().add(new MutableObject<>(inputExpr));
AbstractFunctionCallExpression comparison = new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(not ? AsterixBuiltinFunctions.EQ : AsterixBuiltinFunctions.NEQ));
@@ -1648,8 +1706,8 @@
while (inputOpRefIterator.hasNext()) {
// Generates the variable triple <leftVar, rightVar, outputVar> .
topUnionVar = context.newVar();
- Triple<LogicalVariable, LogicalVariable, LogicalVariable> varTriple =
- new Triple<>(leftInputVar, inputVarIterator.next(), topUnionVar);
+ Triple<LogicalVariable, LogicalVariable, LogicalVariable> varTriple = new Triple<>(leftInputVar,
+ inputVarIterator.next(), topUnionVar);
List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varTriples = new ArrayList<>();
varTriples.add(varTriple);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
index 63d1908..ea3ec2c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
@@ -71,7 +71,6 @@
}
}
- @SuppressWarnings("unchecked")
private static ScalarFunctionCallExpression createFieldAccessExpression(ILogicalExpression target,
List<String> field) {
FunctionIdentifier functionIdentifier;
diff --git a/asterixdb/asterix-algebra/src/main/javacc/AQLPlus.jj b/asterixdb/asterix-algebra/src/main/javacc/AQLPlus.jj
index e0e9388..725f50a 100644
--- a/asterixdb/asterix-algebra/src/main/javacc/AQLPlus.jj
+++ b/asterixdb/asterix-algebra/src/main/javacc/AQLPlus.jj
@@ -41,7 +41,6 @@
import java.util.List;
import java.util.Map;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.aql.clause.DistinctClause;
@@ -100,6 +99,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index cd088c1..5728947 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -27,6 +27,9 @@
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
import org.apache.asterix.api.common.Job.SubmissionMode;
+import org.apache.asterix.app.cc.CompilerExtensionManager;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.app.SessionConfig;
import org.apache.asterix.common.config.AsterixCompilerProperties;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.OptimizationConfUtil;
@@ -50,10 +53,10 @@
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.om.util.AsterixAppContextInfo;
import org.apache.asterix.optimizer.base.RuleCollections;
-import org.apache.asterix.result.ResultUtils;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -89,19 +92,21 @@
* to Hyracks through the Hyracks client interface.
*/
public class APIFramework {
- public static final String HTML_STATEMENT_SEPARATOR = "<!-- BEGIN -->";
private final IRewriterFactory rewriterFactory;
private final IAstPrintVisitorFactory astPrintVisitorFactory;
private final ILangExpressionToPlanTranslatorFactory translatorFactory;
+ private final CompilerExtensionManager cExtensionManager;
- public APIFramework(ILangCompilationProvider compilationProvider) {
+ public APIFramework(ILangCompilationProvider compilationProvider, CompilerExtensionManager cExtensionManager) {
this.rewriterFactory = compilationProvider.getRewriterFactory();
this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
+ this.cExtensionManager = cExtensionManager;
}
- private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultLogicalRewrites() {
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>
+ buildDefaultLogicalRewrites(CompilerExtensionManager ccExtensionManager) {
List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultLogicalRewrites = new ArrayList<>();
SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
@@ -109,14 +114,16 @@
defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildInitialTranslationRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildTypeInferenceRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildAutogenerateIDRuleCollection()));
- defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildNormalizationRuleCollection()));
+ defaultLogicalRewrites
+ .add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildNormalizationRuleCollection(ccExtensionManager)));
defaultLogicalRewrites
.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildCondPushDownAndJoinInferenceRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildLoadFieldsRuleCollection()));
// fj
defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildFuzzyJoinRuleCollection()));
//
- defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildNormalizationRuleCollection()));
+ defaultLogicalRewrites
+ .add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildNormalizationRuleCollection(ccExtensionManager)));
defaultLogicalRewrites
.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildCondPushDownAndJoinInferenceRuleCollection()));
defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildLoadFieldsRuleCollection()));
@@ -251,7 +258,7 @@
HeuristicCompilerFactoryBuilder builder =
new HeuristicCompilerFactoryBuilder(AqlOptimizationContextFactory.INSTANCE);
builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
- builder.setLogicalRewrites(buildDefaultLogicalRewrites());
+ builder.setLogicalRewrites(buildDefaultLogicalRewrites(cExtensionManager));
builder.setPhysicalRewrites(buildDefaultPhysicalRewrites());
IDataFormat format = queryMetadataProvider.getFormat();
ICompilerFactory compilerFactory = builder.create();
@@ -289,7 +296,7 @@
try {
LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
PlanPrettyPrinter.printPlan(plan, pvisitor, 0);
- ResultUtils.displayResults(pvisitor.get().toString(), conf, new ResultUtils.Stats(), null);
+ ResultUtil.displayResults(pvisitor.get().toString(), conf, new Stats(), null);
return null;
} catch (IOException e) {
throw new AlgebricksException(e);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index 7ac5036..265025f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.api.common;
+import org.apache.asterix.app.nc.AsterixNCAppRuntimeContext;
import org.apache.asterix.common.api.AsterixThreadExecutor;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
@@ -33,9 +34,9 @@
public class AsterixAppRuntimeContextProviderForRecovery implements IAsterixAppRuntimeContextProvider {
- private final AsterixAppRuntimeContext asterixAppRuntimeContext;
+ private final AsterixNCAppRuntimeContext asterixAppRuntimeContext;
- public AsterixAppRuntimeContextProviderForRecovery(AsterixAppRuntimeContext asterixAppRuntimeContext) {
+ public AsterixAppRuntimeContextProviderForRecovery(AsterixNCAppRuntimeContext asterixAppRuntimeContext) {
this.asterixAppRuntimeContext = asterixAppRuntimeContext;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 4b00195..b1bf418 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -24,6 +24,8 @@
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import org.apache.asterix.common.config.AsterixPropertiesAccessor;
import org.apache.asterix.common.config.GlobalConfig;
@@ -42,7 +44,7 @@
import org.apache.hyracks.control.nc.NodeControllerService;
public class AsterixHyracksIntegrationUtil {
-
+ private static final Logger LOGGER = Logger.getLogger(AsterixHyracksIntegrationUtil.class.getName());
private static final String IO_DIR_KEY = "java.io.tmpdir";
public static final int DEFAULT_HYRACKS_CC_CLIENT_PORT = 1098;
public static final int DEFAULT_HYRACKS_CC_CLUSTER_PORT = 1099;
@@ -75,7 +77,7 @@
try {
nodeControllerService.start();
} catch (Exception e) {
- e.printStackTrace();
+ LOGGER.log(Level.SEVERE, e.getMessage(), e);
}
}
};
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java
index 204ff5c..b811172 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.api.http.servlet;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
import java.awt.image.BufferedImage;
import java.io.BufferedReader;
import java.io.IOException;
@@ -35,40 +38,38 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.api.common.APIFramework;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.common.app.SessionConfig.OutputFormat;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
import org.apache.asterix.lang.aql.parser.TokenMgrError;
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.result.ResultReader;
-import org.apache.asterix.result.ResultUtils;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.dataset.IHyracksDataset;
import org.apache.hyracks.client.dataset.HyracksDataset;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
public class APIServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
-
private static final Logger LOGGER = Logger.getLogger(APIServlet.class.getName());
+ public static final String HTML_STATEMENT_SEPARATOR = "<!-- BEGIN -->";
private final ILangCompilationProvider aqlCompilationProvider;
private final ILangCompilationProvider sqlppCompilationProvider;
+ private final transient IStatementExecutorFactory statementExectorFactory;
- public APIServlet() {
- this.aqlCompilationProvider = new AqlCompilationProvider();
- this.sqlppCompilationProvider = new SqlppCompilationProvider();
+ public APIServlet(ILangCompilationProvider aqlCompilationProvider,
+ ILangCompilationProvider sqlppCompilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+ this.aqlCompilationProvider = aqlCompilationProvider;
+ this.sqlppCompilationProvider = sqlppCompilationProvider;
+ this.statementExectorFactory = statementExecutorFactory;
}
@Override
@@ -124,20 +125,21 @@
sessionConfig.setOOBData(isSet(printExprParam), isSet(printRewrittenExprParam),
isSet(printLogicalPlanParam), isSet(printOptimizedLogicalPlanParam), isSet(printJob));
MetadataManager.INSTANCE.init();
- QueryTranslator translator = new QueryTranslator(aqlStatements, sessionConfig, compilationProvider);
+ IStatementExecutor translator =
+ statementExectorFactory.create(aqlStatements, sessionConfig, compilationProvider);
double duration = 0;
long startTime = System.currentTimeMillis();
- translator.compileAndExecute(hcc, hds, QueryTranslator.ResultDelivery.SYNC);
+ translator.compileAndExecute(hcc, hds, IStatementExecutor.ResultDelivery.SYNC);
long endTime = System.currentTimeMillis();
duration = (endTime - startTime) / 1000.00;
- out.println(APIFramework.HTML_STATEMENT_SEPARATOR);
+ out.println(HTML_STATEMENT_SEPARATOR);
out.println("<PRE>Duration of all jobs: " + duration + " sec</PRE>");
} catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, pe.toString(), pe);
- ResultUtils.webUIParseExceptionHandler(out, pe, query);
+ ResultUtil.webUIParseExceptionHandler(out, pe, query);
} catch (Exception e) {
GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
- ResultUtils.webUIErrorHandler(out, e);
+ ResultUtil.webUIErrorHandler(out, e);
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/AQLAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/AQLAPIServlet.java
index de19715..bbb9dde 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/AQLAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/AQLAPIServlet.java
@@ -18,21 +18,22 @@
*/
package org.apache.asterix.api.http.servlet;
-import java.util.List;
-
import javax.servlet.http.HttpServletRequest;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
public class AQLAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
private static final String AQL_STMT_PARAM_NAME = "aql";
- private static final List<Byte> ALLOWED_STATEMENTS = Statement.KINDS;
+ private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
+ | Statement.Category.DDL | Statement.Category.PROCEDURE;
- public AQLAPIServlet(ILangCompilationProvider compilationProvider) {
- super(compilationProvider);
+ public AQLAPIServlet(ILangCompilationProvider compilationProvider,
+ IStatementExecutorFactory statementExecutorFactory) {
+ super(compilationProvider, statementExecutorFactory);
}
@Override
@@ -41,13 +42,12 @@
}
@Override
- protected List<Byte> getAllowedStatements() {
- return ALLOWED_STATEMENTS;
+ protected byte getAllowedCategories() {
+ return ALLOWED_CATEGORIES;
}
@Override
protected String getErrorMessage() {
throw new IllegalStateException();
}
-
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
index 8f3b14b..eb23902 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
@@ -25,8 +25,8 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
+import org.apache.asterix.app.result.ResultUtil;
import org.apache.asterix.om.util.AsterixClusterProperties;
-import org.apache.asterix.result.ResultUtils;
import org.json.JSONException;
import org.json.JSONObject;
@@ -43,7 +43,7 @@
responseWriter.write(responseObject.toString());
response.setStatus(HttpServletResponse.SC_OK);
} catch (JSONException e) {
- ResultUtils.apiErrorHandler(responseWriter, e);
+ ResultUtil.apiErrorHandler(responseWriter, e);
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
}
responseWriter.flush();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
index bca6d35..c43377a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
@@ -19,6 +19,8 @@
package org.apache.asterix.api.http.servlet;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
import java.io.IOException;
import java.io.PrintWriter;
import java.util.List;
@@ -43,8 +45,6 @@
import org.json.JSONException;
import org.json.JSONObject;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
/***
* The REST API that takes a dataverse name and a dataset name as the input
* and returns an array of file splits (IP, file-path) of the dataset in LOSSLESS_JSON.
@@ -94,8 +94,8 @@
return;
}
boolean temp = dataset.getDatasetDetails().isTemp();
- FileSplit[] fileSplits = metadataProvider.splitsForDataset(mdTxnCtx, dataverseName, datasetName,
- datasetName, temp);
+ FileSplit[] fileSplits =
+ metadataProvider.splitsForDataset(mdTxnCtx, dataverseName, datasetName, datasetName, temp);
ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
dataset.getItemTypeName());
List<List<String>> primaryKeys = DatasetUtils.getPartitioningKeys(dataset);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java
index 943691d..19a7609 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java
@@ -18,29 +18,20 @@
*/
package org.apache.asterix.api.http.servlet;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
import javax.servlet.http.HttpServletRequest;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
public class DDLAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
+ private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
+ | Statement.Category.DDL;
- private static final List<Byte> allowedStatements = Collections.unmodifiableList(Arrays.asList(
- Statement.Kind.DATAVERSE_DECL, Statement.Kind.DATAVERSE_DROP, Statement.Kind.DATASET_DECL,
- Statement.Kind.NODEGROUP_DECL, Statement.Kind.NODEGROUP_DROP, Statement.Kind.TYPE_DECL,
- Statement.Kind.TYPE_DROP, Statement.Kind.CREATE_INDEX, Statement.Kind.INDEX_DECL,
- Statement.Kind.CREATE_DATAVERSE, Statement.Kind.DATASET_DROP, Statement.Kind.INDEX_DROP,
- Statement.Kind.CREATE_FUNCTION, Statement.Kind.FUNCTION_DROP, Statement.Kind.CREATE_PRIMARY_FEED,
- Statement.Kind.CREATE_SECONDARY_FEED, Statement.Kind.DROP_FEED, Statement.Kind.CREATE_FEED_POLICY,
- Statement.Kind.DROP_FEED_POLICY));
-
- public DDLAPIServlet(ILangCompilationProvider compilationProvider) {
- super(compilationProvider);
+ public DDLAPIServlet(ILangCompilationProvider compilationProvider,
+ IStatementExecutorFactory statementExecutorFactory) {
+ super(compilationProvider, statementExecutorFactory);
}
@Override
@@ -49,8 +40,8 @@
}
@Override
- protected List<Byte> getAllowedStatements() {
- return allowedStatements;
+ protected byte getAllowedCategories() {
+ return ALLOWED_CATEGORIES;
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java
index 63aa2df..cc70324 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java
@@ -18,24 +18,19 @@
*/
package org.apache.asterix.api.http.servlet;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
import javax.servlet.http.HttpServletRequest;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
public class QueryAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
+ private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
- private static final List<Byte> allowedStatements = Collections.unmodifiableList(Arrays.asList(
- Statement.Kind.DATAVERSE_DECL, Statement.Kind.FUNCTION_DECL, Statement.Kind.QUERY, Statement.Kind.SET,
- Statement.Kind.WRITE, Statement.Kind.RUN));
-
- public QueryAPIServlet(ILangCompilationProvider compilationProvider) {
- super(compilationProvider);
+ public QueryAPIServlet(ILangCompilationProvider compilationProvider,
+ IStatementExecutorFactory queryTranslatorFactory) {
+ super(compilationProvider, queryTranslatorFactory);
}
@Override
@@ -44,8 +39,8 @@
}
@Override
- protected List<Byte> getAllowedStatements() {
- return allowedStatements;
+ protected byte getAllowedCategories() {
+ return ALLOWED_CATEGORIES;
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java
index 9d29adf..fdbf253 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.api.http.servlet;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
import java.io.IOException;
import java.io.PrintWriter;
@@ -26,9 +29,10 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.result.ResultReader;
-import org.apache.asterix.result.ResultUtils;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.hyracks.api.client.HyracksConnection;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.dataset.IHyracksDataset;
@@ -38,9 +42,6 @@
import org.json.JSONArray;
import org.json.JSONObject;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
public class QueryResultAPIServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
@@ -77,7 +78,7 @@
JobId jobId = new JobId(handle.getLong(0));
ResultSetId rsId = new ResultSetId(handle.getLong(1));
- ResultReader resultReader = new ResultReader(hcc, hds);
+ ResultReader resultReader = new ResultReader(hds);
resultReader.open(jobId, rsId);
// QQQ The output format is determined by the initial
@@ -87,7 +88,7 @@
// originally determined there. Need to save this value on
// some object that we can obtain here.
SessionConfig sessionConfig = RESTAPIServlet.initResponse(request, response);
- ResultUtils.displayResults(resultReader, sessionConfig, new ResultUtils.Stats(), null);
+ ResultUtil.displayResults(resultReader, sessionConfig, new Stats(), null);
} catch (Exception e) {
out.println(e.getMessage());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java
index 90e8adf..5b91f2f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.api.http.servlet;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
@@ -33,19 +36,20 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.aql.translator.QueryTranslator;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.app.SessionConfig;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.utils.JSONUtil;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
import org.apache.asterix.lang.aql.parser.TokenMgrError;
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.result.ResultReader;
-import org.apache.asterix.result.ResultUtils;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
@@ -53,15 +57,18 @@
import org.apache.hyracks.api.dataset.IHyracksDataset;
import org.apache.hyracks.client.dataset.HyracksDataset;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
public class QueryServiceServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
private static final Logger LOGGER = Logger.getLogger(QueryServiceServlet.class.getName());
+ private final transient ILangCompilationProvider compilationProvider;
+ private final transient IStatementExecutorFactory statementExecutorFactory;
- private transient final ILangCompilationProvider compilationProvider;
+ public QueryServiceServlet(ILangCompilationProvider compilationProvider,
+ IStatementExecutorFactory statementExecutorFactory) {
+ this.compilationProvider = compilationProvider;
+ this.statementExecutorFactory = statementExecutorFactory;
+ }
public enum Parameter {
// Standard
@@ -210,10 +217,6 @@
}
}
- public QueryServiceServlet(final ILangCompilationProvider compilationProvider) {
- this.compilationProvider = compilationProvider;
- }
-
private static String getParameterValue(String content, String attribute) {
if (content == null || attribute == null) {
return null;
@@ -396,12 +399,12 @@
response.setContentType(MediaType.JSON.str());
int respCode = HttpServletResponse.SC_OK;
- ResultUtils.Stats stats = new ResultUtils.Stats();
+ Stats stats = new Stats();
long execStart = 0;
long execEnd = -1;
resultWriter.print("{\n");
- UUID requestId = printRequestId(resultWriter);
+ printRequestId(resultWriter);
printSignature(resultWriter);
printType(resultWriter, sessionConfig);
try {
@@ -419,7 +422,8 @@
IParser parser = compilationProvider.getParserFactory().createParser(query);
List<Statement> aqlStatements = parser.parse();
MetadataManager.INSTANCE.init();
- QueryTranslator translator = new QueryTranslator(aqlStatements, sessionConfig, compilationProvider);
+ IStatementExecutor translator =
+ statementExecutorFactory.create(aqlStatements, sessionConfig, compilationProvider);
execStart = System.nanoTime();
translator.compileAndExecute(hcc, hds, QueryTranslator.ResultDelivery.SYNC, stats);
execEnd = System.nanoTime();
@@ -439,7 +443,8 @@
execEnd = System.nanoTime();
}
}
- printMetrics(resultWriter, System.nanoTime() - elapsedStart, execEnd - execStart, stats.count, stats.size);
+ printMetrics(resultWriter, System.nanoTime() - elapsedStart, execEnd - execStart, stats.getCount(),
+ stats.getSize());
resultWriter.print("}\n");
resultWriter.flush();
String result = stringWriter.toString();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java
index 2eb5e81..9555e11 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.api.http.servlet;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
import java.io.IOException;
import java.io.PrintWriter;
@@ -26,18 +29,14 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.json.JSONArray;
-import org.json.JSONObject;
-
-import org.apache.asterix.result.ResultReader;
+import org.apache.asterix.app.result.ResultReader;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.dataset.IHyracksDataset;
import org.apache.hyracks.api.dataset.ResultSetId;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.client.dataset.HyracksDataset;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+import org.json.JSONArray;
+import org.json.JSONObject;
public class QueryStatusAPIServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
@@ -70,7 +69,7 @@
/* TODO(madhusudancs): We need to find a way to LOSSLESS_JSON serialize default format obtained from
* metadataProvider in the AQLTranslator and store it as part of the result handle.
*/
- ResultReader resultReader = new ResultReader(hcc, hds);
+ ResultReader resultReader = new ResultReader(hds);
resultReader.open(jobId, rsId);
JSONObject jsonResponse = new JSONObject();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java
index 81bf21d..100efed 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.api.http.servlet;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
import java.io.IOException;
import java.io.StringWriter;
import java.nio.charset.StandardCharsets;
@@ -30,9 +33,11 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.common.app.SessionConfig.OutputFormat;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -41,26 +46,26 @@
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.result.ResultReader;
-import org.apache.asterix.result.ResultUtils;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.commons.io.IOUtils;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.dataset.IHyracksDataset;
import org.apache.hyracks.client.dataset.HyracksDataset;
import org.json.JSONObject;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
abstract class RESTAPIServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
private final ILangCompilationProvider compilationProvider;
private final IParserFactory parserFactory;
+ private final transient IStatementExecutorFactory statementExecutorFactory;
- public RESTAPIServlet(ILangCompilationProvider compilationProvider) {
+ public RESTAPIServlet(ILangCompilationProvider compilationProvider,
+ IStatementExecutorFactory statementExecutorFactory) {
this.compilationProvider = compilationProvider;
this.parserFactory = compilationProvider.getParserFactory();
+ this.statementExecutorFactory = statementExecutorFactory;
}
/**
@@ -193,26 +198,27 @@
List<Statement> aqlStatements = parser.parse();
if (!containsForbiddenStatements(aqlStatements)) {
MetadataManager.INSTANCE.init();
- QueryTranslator translator = new QueryTranslator(aqlStatements, sessionConfig, compilationProvider);
+ IStatementExecutor translator =
+ statementExecutorFactory.create(aqlStatements, sessionConfig, compilationProvider);
translator.compileAndExecute(hcc, hds, resultDelivery);
}
} catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, pe.getMessage(), pe);
- String errorMessage = ResultUtils.buildParseExceptionMessage(pe, query);
+ String errorMessage = ResultUtil.buildParseExceptionMessage(pe, query);
JSONObject errorResp =
- ResultUtils.getErrorResponse(2, errorMessage, "", ResultUtils.extractFullStackTrace(pe));
+ ResultUtil.getErrorResponse(2, errorMessage, "", ResultUtil.extractFullStackTrace(pe));
sessionConfig.out().write(errorResp.toString());
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
} catch (Exception e) {
GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
- ResultUtils.apiErrorHandler(sessionConfig.out(), e);
+ ResultUtil.apiErrorHandler(sessionConfig.out(), e);
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
}
}
private boolean containsForbiddenStatements(List<Statement> aqlStatements) throws AsterixException {
for (Statement st : aqlStatements) {
- if (!getAllowedStatements().contains(st.getKind())) {
+ if ((st.getCategory() & getAllowedCategories()) == 0) {
throw new AsterixException(String.format(getErrorMessage(), st.getKind()));
}
}
@@ -233,7 +239,7 @@
protected abstract String getQueryParameter(HttpServletRequest request);
- protected abstract List<Byte> getAllowedStatements();
+ protected abstract byte getAllowedCategories();
protected abstract String getErrorMessage();
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java
index afa202b..1c62cdb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java
@@ -18,25 +18,19 @@
*/
package org.apache.asterix.api.http.servlet;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
import javax.servlet.http.HttpServletRequest;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
public class UpdateAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
- private static final List<Byte> allowedStatements = Collections.unmodifiableList(Arrays.asList(
- Statement.Kind.DATAVERSE_DECL, Statement.Kind.DELETE, Statement.Kind.INSERT,
- Statement.Kind.UPSERT, Statement.Kind.UPDATE, Statement.Kind.DML_CMD_LIST, Statement.Kind.LOAD,
- Statement.Kind.CONNECT_FEED, Statement.Kind.DISCONNECT_FEED, Statement.Kind.SET,
- Statement.Kind.COMPACT, Statement.Kind.EXTERNAL_DATASET_REFRESH, Statement.Kind.RUN));
+ private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
- public UpdateAPIServlet(ILangCompilationProvider compilationProvider) {
- super(compilationProvider);
+ public UpdateAPIServlet(ILangCompilationProvider compilationProvider,
+ IStatementExecutorFactory statementExecutorFactory) {
+ super(compilationProvider, statementExecutorFactory);
}
@Override
@@ -45,8 +39,8 @@
}
@Override
- protected List<Byte> getAllowedStatements() {
- return allowedStatements;
+ protected byte getAllowedCategories() {
+ return ALLOWED_CATEGORIES;
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index 964893b..e358963 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -24,14 +24,16 @@
import org.apache.asterix.api.common.APIFramework;
import org.apache.asterix.api.common.Job;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.common.app.SessionConfig.OutputFormat;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.job.JobSpecification;
@@ -46,20 +48,22 @@
private final ILangCompilationProvider compilationProvider;
private final IParserFactory parserFactory;
private final APIFramework apiFramework;
+ private final IStatementExecutorFactory statementExecutorFactory;
public AsterixJavaClient(IHyracksClientConnection hcc, Reader queryText, PrintWriter writer,
- ILangCompilationProvider compilationProvider) {
+ ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
this.hcc = hcc;
this.queryText = queryText;
this.writer = writer;
this.compilationProvider = compilationProvider;
+ this.apiFramework = new APIFramework(compilationProvider, null);
+ this.statementExecutorFactory = statementExecutorFactory;
parserFactory = compilationProvider.getParserFactory();
- this.apiFramework = new APIFramework(compilationProvider);
}
public AsterixJavaClient(IHyracksClientConnection hcc, Reader queryText,
- ILangCompilationProvider compilationProvider) {
- this(hcc, queryText, new PrintWriter(System.out, true), compilationProvider);
+ ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+ this(hcc, queryText, new PrintWriter(System.out, true), compilationProvider, statementExecutorFactory);
}
public void compile() throws Exception {
@@ -68,7 +72,7 @@
public void compile(boolean optimize, boolean printRewrittenExpressions, boolean printLogicalPlan,
boolean printOptimizedPlan, boolean printPhysicalOpsOnly, boolean generateBinaryRuntime, boolean printJob)
- throws Exception {
+ throws Exception {
queryJobSpec = null;
dmlJobs = null;
@@ -90,7 +94,7 @@
conf.set(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS, true);
}
- QueryTranslator translator = new QueryTranslator(statements, conf, compilationProvider);
+ IStatementExecutor translator = statementExecutorFactory.create(statements, conf, compilationProvider);
translator.compileAndExecute(hcc, null, QueryTranslator.ResultDelivery.SYNC);
writer.flush();
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
new file mode 100644
index 0000000..af10394
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
@@ -0,0 +1,156 @@
+/*
+ * 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.app.cc;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.algebra.base.ILangExtension.Language;
+import org.apache.asterix.algebra.extension.ExtensionFunctionIdentifier;
+import org.apache.asterix.algebra.extension.IAlgebraExtensionManager;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.common.api.ExtensionId;
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.compiler.provider.AqlCompilationProvider;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * AsterixDB's implementation of {@code IAlgebraExtensionManager} which takes care of
+ * initializing extensions for App and Compilation purposes
+ */
+public class CompilerExtensionManager implements IAlgebraExtensionManager {
+
+ private static final String ERROR_MESSAGE_ID_CONFLICT = "Two Extensions share the same Id: %1$s";
+ public static final String ERROR_MESSAGE_COMPONENT_CONFLICT =
+ "Extension Conflict between %1$s and %2$s both extensions extend %3$s";
+
+ private final Map<ExtensionId, IExtension> extensions = new HashMap<>();
+
+ private final IStatementExecutorExtension statementExecutorExtension;
+ private final ILangCompilationProvider aqlCompilationProvider;
+ private final ILangCompilationProvider sqlppCompilationProvider;
+ private final DefaultStatementExecutorFactory defaultQueryTranslatorFactory;
+
+ /**
+ * Initialize {@code CompilerExtensionManager} from configuration
+ *
+ * @param list
+ * @throws InstantiationException
+ * @throws IllegalAccessException
+ * @throws ClassNotFoundException
+ * @throws HyracksDataException
+ */
+ public CompilerExtensionManager(List<AsterixExtension> list)
+ throws InstantiationException, IllegalAccessException, ClassNotFoundException, HyracksDataException {
+ Pair<ExtensionId, ILangCompilationProvider> aqlcp = null;
+ Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
+ IStatementExecutorExtension see = null;
+ defaultQueryTranslatorFactory = new DefaultStatementExecutorFactory(this);
+
+ if (list != null) {
+ for (AsterixExtension extensionConf : list) {
+ IExtension extension = (IExtension) Class.forName(extensionConf.getClassName()).newInstance();
+ extension.configure(extensionConf.getArgs());
+ if (extensions.containsKey(extension.getId())) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
+ ERROR_MESSAGE_ID_CONFLICT, extension.getId());
+ }
+ extensions.put(extension.getId(), extension);
+ switch (extension.getExtensionKind()) {
+ case STATEMENT_EXECUTOR:
+ see = extendStatementExecutor(see, (IStatementExecutorExtension) extension);
+ break;
+ case LANG:
+ ILangExtension le = (ILangExtension) extension;
+ aqlcp = extendLangCompilationProvider(Language.AQL, aqlcp, le);
+ sqlppcp = extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
+ break;
+ default:
+ break;
+ }
+ }
+ }
+ this.statementExecutorExtension = see;
+ this.aqlCompilationProvider = aqlcp == null ? new AqlCompilationProvider() : aqlcp.second;
+ this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
+ }
+
+ private Pair<ExtensionId, ILangCompilationProvider> extendLangCompilationProvider(Language lang,
+ Pair<ExtensionId, ILangCompilationProvider> cp, ILangExtension le) throws HyracksDataException {
+ if (cp != null && le.getLangCompilationProvider(lang) != null) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
+ ERROR_MESSAGE_COMPONENT_CONFLICT, le.getId(), cp.first, lang.toString());
+ }
+ return (le.getLangCompilationProvider(lang) != null)
+ ? new Pair<>(le.getId(), le.getLangCompilationProvider(lang)) : cp;
+ }
+
+ private IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension qte,
+ IStatementExecutorExtension extension) throws HyracksDataException {
+ if (qte != null) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
+ ERROR_MESSAGE_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
+ IStatementExecutorFactory.class.getSimpleName());
+ }
+ return extension;
+ }
+
+ public IStatementExecutorFactory getQueryTranslatorFactory() {
+ return statementExecutorExtension == null ? defaultQueryTranslatorFactory
+ : statementExecutorExtension.getQueryTranslatorFactory();
+ }
+
+ public ILangCompilationProvider getAqlCompilationProvider() {
+ return aqlCompilationProvider;
+ }
+
+ public ILangCompilationProvider getSqlppCompilationProvider() {
+ return sqlppCompilationProvider;
+ }
+
+ // TODO(amoudi/yingyi): this is not a good way to extend rewrite rules. introduce re-write rule provider
+ @Override
+ public boolean unnestToDataScan(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ UnnestOperator unnestOp, ILogicalExpression unnestExpr, AbstractFunctionCallExpression functionCallExpr)
+ throws AlgebricksException {
+ FunctionIdentifier functionId = functionCallExpr.getFunctionIdentifier();
+ if (functionId instanceof ExtensionFunctionIdentifier) {
+ ILangExtension extension =
+ (ILangExtension) extensions.get(((ExtensionFunctionIdentifier) functionId).getExtensionId());
+ return extension.unnestToDataScan(opRef, context, unnestOp, unnestExpr, functionCallExpr);
+ }
+ return false;
+ }
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java
similarity index 60%
copy from asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java
index 8b0914a..a4b2345 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java
@@ -16,24 +16,22 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.active;
+package org.apache.asterix.app.cc;
-public class ActiveRuntime implements IActiveRuntime {
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.translator.IStatementExecutorFactory;
- /** A unique identifier for the runtime **/
- protected final ActiveRuntimeId runtimeId;
-
- public ActiveRuntime(ActiveRuntimeId runtimeId) {
- this.runtimeId = runtimeId;;
- }
-
+/**
+ * An interface for extensions of {@code IStatementExecutor}
+ */
+public interface IStatementExecutorExtension extends IExtension {
@Override
- public ActiveRuntimeId getRuntimeId() {
- return runtimeId;
+ default ExtensionKind getExtensionKind() {
+ return ExtensionKind.STATEMENT_EXECUTOR;
}
- @Override
- public String toString() {
- return runtimeId.toString();
- }
+ /**
+ * @return The extension implementation of the {@code IStatementExecutorFactory}
+ */
+ IStatementExecutorFactory getQueryTranslatorFactory();
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ActiveLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ActiveLifecycleListener.java
index c44ffc1..fe15ce8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ActiveLifecycleListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ActiveLifecycleListener.java
@@ -26,7 +26,8 @@
import org.apache.asterix.active.ActiveEvent;
import org.apache.asterix.active.ActiveJobNotificationHandler;
-import org.apache.asterix.active.ActivePartitionMessage;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.message.ActivePartitionMessage;
import org.apache.hyracks.api.exceptions.HyracksException;
import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import org.apache.hyracks.api.job.IJobLifecycleListener;
@@ -48,15 +49,17 @@
@Override
public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
- if (ActiveJobNotificationHandler.INSTANCE.isActiveJob(jobId)) {
- jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_START));
+ EntityId entityId = ActiveJobNotificationHandler.INSTANCE.getEntity(jobId);
+ if (entityId != null) {
+ jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_START, entityId));
}
}
@Override
public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
- if (ActiveJobNotificationHandler.INSTANCE.isActiveJob(jobId)) {
- jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_FINISH));
+ EntityId entityId = ActiveJobNotificationHandler.INSTANCE.getEntity(jobId);
+ if (entityId != null) {
+ jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_FINISH, entityId));
} else {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
@@ -72,7 +75,7 @@
public void receive(ActivePartitionMessage message) {
if (ActiveJobNotificationHandler.INSTANCE.isActiveJob(message.getJobId())) {
jobEventInbox.add(new ActiveEvent(message.getJobId(), ActiveEvent.EventKind.PARTITION_EVENT,
- message.getFeedId(), message.getPayload()));
+ message.getActiveRuntimeId().getEntityId(), message));
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
index 92ef062..95fe68c 100755
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
@@ -265,8 +265,8 @@
String adapterFactoryClass = adapter.getFactoryClass().trim();
String adapterName = libraryName + "#" + adapter.getName().trim();
AdapterIdentifier aid = new AdapterIdentifier(dataverse, adapterName);
- DatasourceAdapter dsa = new DatasourceAdapter(aid, adapterFactoryClass,
- IDataSourceAdapter.AdapterType.EXTERNAL);
+ DatasourceAdapter dsa =
+ new DatasourceAdapter(aid, adapterFactoryClass, IDataSourceAdapter.AdapterType.EXTERNAL);
MetadataManager.INSTANCE.addAdapter(mdTxnCtx, dsa);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Installed adapter: " + adapterName);
@@ -334,8 +334,8 @@
}
// get a reference to the specific library dir
- File libDir = new File(
- installDir.getAbsolutePath() + File.separator + dataverse + File.separator + libraryName);
+ File libDir =
+ new File(installDir.getAbsolutePath() + File.separator + dataverse + File.separator + libraryName);
FilenameFilter jarFileFilter = new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
index ba15fb1..b815602 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
@@ -22,9 +22,10 @@
import java.util.ArrayList;
import java.util.List;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.common.app.SessionConfig.OutputFormat;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.external.feed.api.IFeedWork;
@@ -36,6 +37,7 @@
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.om.util.AsterixAppContextInfo;
+import org.apache.asterix.translator.IStatementExecutor;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
@@ -68,6 +70,7 @@
private static class SubscribeFeedWorkRunnable implements Runnable {
+ private static final DefaultStatementExecutorFactory qtFactory = new DefaultStatementExecutorFactory(null);
private final FeedConnectionRequest request;
private final String[] locations;
@@ -82,13 +85,13 @@
//TODO(amoudi): route PrintWriter to log file
PrintWriter writer = new PrintWriter(System.err, true);
SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
- DataverseDecl dataverseDecl =
- new DataverseDecl(new Identifier(request.getReceivingFeedId().getDataverse()));
+ DataverseDecl dataverseDecl = new DataverseDecl(
+ new Identifier(request.getReceivingFeedId().getDataverse()));
SubscribeFeedStatement subscribeStmt = new SubscribeFeedStatement(locations, request);
List<Statement> statements = new ArrayList<Statement>();
statements.add(dataverseDecl);
statements.add(subscribeStmt);
- QueryTranslator translator = new QueryTranslator(statements, pc, compilationProvider);
+ IStatementExecutor translator = qtFactory.create(statements, pc, compilationProvider);
translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
QueryTranslator.ResultDelivery.SYNC);
if (LOGGER.isEnabledFor(Level.INFO)) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/AsterixNCAppRuntimeContext.java
similarity index 90%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/AsterixNCAppRuntimeContext.java
index 4bb09d4..3ebe873 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/AsterixNCAppRuntimeContext.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.api.common;
+package org.apache.asterix.app.nc;
import java.io.IOException;
import java.rmi.RemoteException;
@@ -26,12 +26,14 @@
import java.util.logging.Logger;
import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.api.common.AsterixAppRuntimeContextProviderForRecovery;
import org.apache.asterix.common.api.AsterixThreadExecutor;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixCompilerProperties;
+import org.apache.asterix.common.config.AsterixExtensionProperties;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.AsterixFeedProperties;
import org.apache.asterix.common.config.AsterixMetadataProperties;
@@ -94,8 +96,8 @@
import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
import org.apache.hyracks.storage.common.file.IResourceIdFactory;
-public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
- private static final Logger LOGGER = Logger.getLogger(AsterixAppRuntimeContext.class.getName());
+public class AsterixNCAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
+ private static final Logger LOGGER = Logger.getLogger(AsterixNCAppRuntimeContext.class.getName());
private ILSMMergePolicyFactory metadataMergePolicyFactory;
private final INCApplicationContext ncApplicationContext;
@@ -129,10 +131,12 @@
private IReplicaResourcesManager replicaResourcesManager;
private final int metadataRmiPort;
- private ILibraryManager libraryManager;
+ private final ILibraryManager libraryManager;
+ private final NCExtensionManager ncExtensionManager;
- public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext, int metadataRmiPort)
- throws AsterixException {
+ public AsterixNCAppRuntimeContext(INCApplicationContext ncApplicationContext, int metadataRmiPort)
+ throws AsterixException, InstantiationException, IllegalAccessException, ClassNotFoundException,
+ IOException {
this.ncApplicationContext = ncApplicationContext;
// Determine whether to use old-style asterix-configuration.xml or new-style configuration.
// QQQ strip this out eventually
@@ -151,9 +155,12 @@
txnProperties = new AsterixTransactionProperties(propertiesAccessor);
feedProperties = new AsterixFeedProperties(propertiesAccessor);
buildProperties = new AsterixBuildProperties(propertiesAccessor);
- replicationProperties =
- new AsterixReplicationProperties(propertiesAccessor, AsterixClusterProperties.INSTANCE.getCluster());
+ replicationProperties = new AsterixReplicationProperties(propertiesAccessor,
+ AsterixClusterProperties.INSTANCE.getCluster());
this.metadataRmiPort = metadataRmiPort;
+ libraryManager = new ExternalLibraryManager();
+ ncExtensionManager = new NCExtensionManager(
+ new AsterixExtensionProperties(propertiesAccessor).getExtensions());
}
@Override
@@ -177,8 +184,8 @@
new PersistentLocalResourceRepositoryFactory(ioManager, ncApplicationContext.getNodeId(),
metadataProperties);
- localResourceRepository =
- (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
+ localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
+ .createRepository();
IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider =
new AsterixAppRuntimeContextProviderForRecovery(this);
@@ -239,16 +246,18 @@
remoteRecoveryManager = new RemoteRecoveryManager(replicationManager, this, replicationProperties);
- bufferCache =
- new BufferCache(ioManager, prs, pcp, fileMapManager, storageProperties.getBufferCacheMaxOpenFiles(),
- ncApplicationContext.getThreadFactory(), replicationManager);
+ bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
+ storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory(),
+ replicationManager);
} else {
bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
}
- // The order of registration is important. The buffer cache must registered before recovery and transaction managers.
- //Notes: registered components are stopped in reversed order
+ /*
+ * The order of registration is important. The buffer cache must registered before recovery and transaction
+ * managers. Notes: registered components are stopped in reversed order
+ */
ILifeCycleComponentManager lccm = ncApplicationContext.getLifeCycleComponentManager();
lccm.register((ILifeCycleComponent) bufferCache);
/**
@@ -270,11 +279,6 @@
lccm.register((ILifeCycleComponent) datasetLifecycleManager);
lccm.register((ILifeCycleComponent) txnSubsystem.getTransactionManager());
lccm.register((ILifeCycleComponent) txnSubsystem.getLockManager());
-
- /**
- * Initializes the library manager.
- */
- libraryManager = new ExternalLibraryManager();
}
@Override
@@ -387,7 +391,7 @@
}
@Override
- public ActiveManager getFeedManager() {
+ public ActiveManager getActiveManager() {
return activeManager;
}
@@ -432,7 +436,8 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Bootstrapping metadata");
}
- MetadataNode.INSTANCE.initialize(this);
+ MetadataNode.INSTANCE.initialize(this, ncExtensionManager.getMetadataTupleTranslatorProvider(),
+ ncExtensionManager.getMetadataExtensions());
proxy = (IAsterixStateProxy) ncApplicationContext.getDistributedState();
if (proxy == null) {
@@ -442,9 +447,10 @@
// This is a special case, we just give the metadataNode directly.
// This way we can delay the registration of the metadataNode until
// it is completely initialized.
- MetadataManager.INSTANCE = new MetadataManager(proxy, MetadataNode.INSTANCE);
+ MetadataManager.instantiate(new MetadataManager(proxy, MetadataNode.INSTANCE));
MetadataBootstrap.startUniverse(this, ncApplicationContext, newUniverse);
MetadataBootstrap.startDDLRecovery();
+ ncExtensionManager.initializeMetadata();
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Metadata node bound");
@@ -462,4 +468,8 @@
UnicastRemoteObject.unexportObject(MetadataNode.INSTANCE, false);
}
+ public NCExtensionManager getNcExtensionManager() {
+ return ncExtensionManager;
+ }
+
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
new file mode 100644
index 0000000..759bf09
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
@@ -0,0 +1,110 @@
+/*
+ * 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.app.nc;
+
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.app.cc.CompilerExtensionManager;
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.api.IMetadataExtension;
+import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * AsterixDB's implementation of {@code INCExtensionManager} which takes care of
+ * initializing extensions on Node Controllers
+ */
+public class NCExtensionManager {
+
+ private final MetadataTupleTranslatorProvider tupleTranslatorProvider;
+ private final List<IMetadataExtension> mdExtensions;
+
+ /**
+ * Initialize {@code CCExtensionManager} from configuration
+ *
+ * @param list
+ * @throws InstantiationException
+ * @throws IllegalAccessException
+ * @throws ClassNotFoundException
+ * @throws HyracksDataException
+ */
+ public NCExtensionManager(List<AsterixExtension> list)
+ throws InstantiationException, IllegalAccessException, ClassNotFoundException, HyracksDataException {
+ MetadataTupleTranslatorProvider ttp = null;
+ IMetadataExtension tupleTranslatorExtension = null;
+ mdExtensions = new ArrayList<>();
+ if (list != null) {
+ for (AsterixExtension extensionConf : list) {
+ IExtension extension = (IExtension) Class.forName(extensionConf.getClassName()).newInstance();
+ extension.configure(extensionConf.getArgs());
+ switch (extension.getExtensionKind()) {
+ case METADATA:
+ IMetadataExtension mde = (IMetadataExtension) extension;
+ mdExtensions.add(mde);
+ ttp = extendTupleTranslator(ttp, tupleTranslatorExtension, mde);
+ tupleTranslatorExtension = ttp == null ? null : mde;
+ break;
+ default:
+ break;
+ }
+ }
+ }
+ this.tupleTranslatorProvider = ttp == null ? new MetadataTupleTranslatorProvider() : ttp;
+ }
+
+ private MetadataTupleTranslatorProvider extendTupleTranslator(MetadataTupleTranslatorProvider ttp,
+ IMetadataExtension tupleTranslatorExtension, IMetadataExtension mde) throws HyracksDataException {
+ if (ttp != null) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
+ CompilerExtensionManager.ERROR_MESSAGE_COMPONENT_CONFLICT, tupleTranslatorExtension.getId(),
+ mde.getId(), IMetadataExtension.class.getSimpleName());
+ }
+ return mde.getMetadataTupleTranslatorProvider();
+ }
+
+ public List<IMetadataExtension> getMetadataExtensions() {
+ return mdExtensions;
+ }
+
+ public MetadataTupleTranslatorProvider getMetadataTupleTranslatorProvider() {
+ return tupleTranslatorProvider;
+ }
+
+ /**
+ * Called on bootstrap of metadata node allowing extensions to instantiate their Metadata artifacts
+ *
+ * @throws HyracksDataException
+ */
+ public void initializeMetadata() throws HyracksDataException {
+ if (mdExtensions != null) {
+ for (IMetadataExtension mdExtension : mdExtensions) {
+ try {
+ mdExtension.initializeMetadata();
+ } catch (RemoteException | ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
similarity index 93%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultPrinter.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
index 8e16ef7..9b1b32b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.result;
+package org.apache.asterix.app.result;
import static java.nio.charset.StandardCharsets.UTF_8;
@@ -24,9 +24,11 @@
import java.io.StringWriter;
import java.nio.ByteBuffer;
-import org.apache.asterix.api.common.SessionConfig;
+import org.apache.asterix.common.app.SessionConfig;
import org.apache.asterix.common.utils.JSONUtil;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
import org.apache.hyracks.api.comm.IFrame;
@@ -41,7 +43,7 @@
private static FrameManager resultDisplayFrameMgr = new FrameManager(ResultReader.FRAME_SIZE);
private final SessionConfig conf;
- private final ResultUtils.Stats stats;
+ private final Stats stats;
private final ARecordType recordType;
private boolean indentJSON;
@@ -52,7 +54,7 @@
// Whether this is the first instance being output
private boolean notFirst = false;
- public ResultPrinter(SessionConfig conf, ResultUtils.Stats stats, ARecordType recordType) {
+ public ResultPrinter(SessionConfig conf, Stats stats, ARecordType recordType) {
this.conf = conf;
this.stats = stats;
this.recordType = recordType;
@@ -142,9 +144,9 @@
record = JSONUtil.quoteAndEscape(record);
}
conf.out().print(record);
- ++stats.count;
+ stats.setCount(stats.getCount() + 1);
// TODO(tillw) fix this approximation
- stats.size += record.length();
+ stats.setSize(stats.getSize() + record.length());
}
public void print(String record) throws HyracksDataException {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
similarity index 92%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultReader.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index 373c0d0..99235be 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -16,10 +16,9 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.result;
+package org.apache.asterix.app.result;
import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.comm.IFrame;
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
@@ -42,7 +41,7 @@
public static final int FRAME_SIZE = AsterixAppContextInfo.getInstance().getCompilerProperties().getFrameSize();
- public ResultReader(IHyracksClientConnection hcc, IHyracksDataset hdc) throws Exception {
+ public ResultReader(IHyracksDataset hdc) {
hyracksDataset = hdc;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
similarity index 81%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultUtils.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
index 3503549..b5fd96e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/result/ResultUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.result;
+package org.apache.asterix.app.result;
import java.io.BufferedReader;
import java.io.IOException;
@@ -24,44 +24,52 @@
import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.io.StringWriter;
-import java.nio.charset.Charset;
-import java.util.HashMap;
+import java.util.AbstractMap;
+import java.util.Collections;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
-import org.apache.asterix.api.common.SessionConfig;
import org.apache.asterix.api.http.servlet.APIServlet;
+import org.apache.asterix.common.app.SessionConfig;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.http.ParseException;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.log4j.Logger;
import org.json.JSONArray;
import org.json.JSONException;
import org.json.JSONObject;
-public class ResultUtils {
- static Map<Character, String> HTML_ENTITIES = new HashMap<Character, String>();
+public class ResultUtil {
+ private static final Logger LOGGER = Logger.getLogger(ResultUtil.class.getName());
+ public static final Map<Character, String> HTML_ENTITIES = Collections.unmodifiableMap(Stream.of(
+ new AbstractMap.SimpleImmutableEntry<>('"', """), new AbstractMap.SimpleImmutableEntry<>('&', "&"),
+ new AbstractMap.SimpleImmutableEntry<>('<', "<"), new AbstractMap.SimpleImmutableEntry<>('>', ">"))
+ .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
- static {
- HTML_ENTITIES.put('"', """);
- HTML_ENTITIES.put('&', "&");
- HTML_ENTITIES.put('<', "<");
- HTML_ENTITIES.put('>', ">");
+ private ResultUtil() {
+
}
- public static class Stats {
- public long count;
- public long size;
- }
-
- public static String escapeHTML(String s) {
- for (Character c : HTML_ENTITIES.keySet()) {
- if (s.indexOf(c) >= 0) {
- s = s.replace(c.toString(), HTML_ENTITIES.get(c));
+ /**
+ * escapes html entities in aString
+ *
+ * @param aString
+ * @return escaped String
+ */
+ public static String escapeHTML(String aString) {
+ String escaped = aString;
+ for (Entry<Character, String> entry : HTML_ENTITIES.entrySet()) {
+ if (escaped.indexOf(entry.getKey()) >= 0) {
+ escaped = escaped.replace(entry.getKey().toString(), entry.getValue());
}
}
- return s;
+ return escaped;
}
public static void displayResults(ResultReader resultReader, SessionConfig conf, Stats stats,
@@ -90,6 +98,7 @@
}
errorResp.put("stacktrace", errorStackTrace);
} catch (JSONException e) {
+ LOGGER.warn("Failed to build the result's JSON object", e);
// TODO(madhusudancs): Figure out what to do when JSONException occurs while building the results.
}
return errorResp;
@@ -120,7 +129,7 @@
errorCode = 4;
}
- JSONObject errorResp = ResultUtils.getErrorResponse(errorCode, extractErrorMessage(e), extractErrorSummary(e),
+ JSONObject errorResp = ResultUtil.getErrorResponse(errorCode, extractErrorMessage(e), extractErrorSummary(e),
extractFullStackTrace(e));
out.write(errorResp.toString());
}
@@ -150,12 +159,13 @@
}
private static Throwable getRootCause(Throwable cause) {
+ Throwable currentCause = cause;
Throwable nextCause = cause.getCause();
- while (nextCause != null) {
- cause = nextCause;
+ while (nextCause != null && nextCause != currentCause) {
+ currentCause = nextCause;
nextCause = cause.getCause();
}
- return cause;
+ return currentCause;
}
/**
@@ -220,6 +230,7 @@
* The default template string if the template file does not exist or is not readable
* @return The template string to be used to render the output.
*/
+ //TODO(till|amoudi|mblow|yingyi|ceej|imaxon): path is ignored completely!!
private static String readTemplateFile(String path, String defaultTemplate) {
String errorTemplate = defaultTemplate;
try {
@@ -236,7 +247,7 @@
}
errorTemplate = sb.toString();
} catch (IOException ioe) {
- // If there is an IOException reading the error template html file, default value of error template is used.
+ LOGGER.warn("Unable to read template error message file", ioe);
}
return errorTemplate;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
new file mode 100644
index 0000000..c802d08
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.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.app.translator;
+
+import java.util.List;
+
+import org.apache.asterix.app.cc.CompilerExtensionManager;
+import org.apache.asterix.common.app.SessionConfig;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+
+public class DefaultStatementExecutorFactory implements IStatementExecutorFactory {
+
+ protected final CompilerExtensionManager cExtensionManager;
+
+ public DefaultStatementExecutorFactory(CompilerExtensionManager cExtensionManager) {
+ this.cExtensionManager = cExtensionManager;
+ }
+
+ @Override
+ public QueryTranslator create(List<Statement> aqlStatements, SessionConfig conf,
+ ILangCompilationProvider compilationProvider) {
+ return new QueryTranslator(aqlStatements, conf, compilationProvider, cExtensionManager);
+ }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
similarity index 85%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index d6065fb..24c678c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.aql.translator;
+package org.apache.asterix.app.translator;
import java.io.BufferedReader;
import java.io.File;
@@ -41,18 +41,22 @@
import org.apache.asterix.active.ActivityState;
import org.apache.asterix.active.EntityId;
import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.algebra.extension.IExtensionStatement;
import org.apache.asterix.api.common.APIFramework;
-import org.apache.asterix.api.common.SessionConfig;
+import org.apache.asterix.api.http.servlet.APIServlet;
+import org.apache.asterix.app.cc.CompilerExtensionManager;
import org.apache.asterix.app.external.ExternalIndexingOperations;
import org.apache.asterix.app.external.FeedJoint;
import org.apache.asterix.app.external.FeedOperations;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.app.SessionConfig;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
@@ -98,7 +102,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
@@ -144,6 +148,7 @@
import org.apache.asterix.metadata.utils.DatasetUtils;
import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.MetadataLockManager;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -152,8 +157,6 @@
import org.apache.asterix.om.util.AsterixAppContextInfo;
import org.apache.asterix.om.util.AsterixClusterProperties;
import org.apache.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
-import org.apache.asterix.result.ResultReader;
-import org.apache.asterix.result.ResultUtils;
import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
import org.apache.asterix.translator.AbstractLangTranslator;
import org.apache.asterix.translator.CompiledStatements.CompiledConnectFeedStatement;
@@ -167,11 +170,14 @@
import org.apache.asterix.translator.CompiledStatements.CompiledSubscribeFeedStatement;
import org.apache.asterix.translator.CompiledStatements.CompiledUpsertStatement;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.TypeTranslator;
import org.apache.asterix.translator.util.ValidateUtil;
import org.apache.asterix.util.FlushDatasetUtils;
import org.apache.asterix.util.JobUtils;
import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -199,39 +205,33 @@
* Provides functionality for executing a batch of Query statements (queries included)
* sequentially.
*/
-public class QueryTranslator extends AbstractLangTranslator {
+public class QueryTranslator extends AbstractLangTranslator implements IStatementExecutor {
private static final Logger LOGGER = Logger.getLogger(QueryTranslator.class.getName());
- private enum ProgressState {
+ protected enum ProgressState {
NO_PROGRESS,
ADDED_PENDINGOP_RECORD_TO_METADATA
}
- public enum ResultDelivery {
- SYNC,
- ASYNC,
- ASYNC_DEFERRED
- }
-
public static final boolean IS_DEBUG_MODE = false;// true
- private final List<Statement> statements;
- private final SessionConfig sessionConfig;
- private Dataverse activeDefaultDataverse;
- private final List<FunctionDecl> declaredFunctions;
- private final APIFramework apiFramework;
- private final IRewriterFactory rewriterFactory;
+ protected final List<Statement> statements;
+ protected final SessionConfig sessionConfig;
+ protected Dataverse activeDefaultDataverse;
+ protected final List<FunctionDecl> declaredFunctions;
+ protected final APIFramework apiFramework;
+ protected final IRewriterFactory rewriterFactory;
public QueryTranslator(List<Statement> aqlStatements, SessionConfig conf,
- ILangCompilationProvider compliationProvider) {
+ ILangCompilationProvider compliationProvider, CompilerExtensionManager ccExtensionManager) {
this.statements = aqlStatements;
this.sessionConfig = conf;
this.declaredFunctions = getDeclaredFunctions(aqlStatements);
- this.apiFramework = new APIFramework(compliationProvider);
+ this.apiFramework = new APIFramework(compliationProvider, ccExtensionManager);
this.rewriterFactory = compliationProvider.getRewriterFactory();
}
- private List<FunctionDecl> getDeclaredFunctions(List<Statement> statements) {
+ protected List<FunctionDecl> getDeclaredFunctions(List<Statement> statements) {
List<FunctionDecl> functionDecls = new ArrayList<>();
for (Statement st : statements) {
if (st.getKind() == Statement.Kind.FUNCTION_DECL) {
@@ -253,13 +253,15 @@
* @return A List<QueryResult> containing a QueryResult instance corresponding to each submitted query.
* @throws Exception
*/
+ @Override
public void compileAndExecute(IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery)
throws Exception {
- compileAndExecute(hcc, hdc, resultDelivery, new ResultUtils.Stats());
+ compileAndExecute(hcc, hdc, resultDelivery, new Stats());
}
+ @Override
public void compileAndExecute(IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery,
- ResultUtils.Stats stats) throws Exception {
+ Stats stats) throws Exception {
int resultSetIdCounter = 0;
FileSplit outputFile = null;
IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
@@ -274,7 +276,7 @@
try {
for (Statement stmt : statements) {
if (sessionConfig.is(SessionConfig.FORMAT_HTML)) {
- sessionConfig.out().println(APIFramework.HTML_STATEMENT_SEPARATOR);
+ sessionConfig.out().println(APIServlet.HTML_STATEMENT_SEPARATOR);
}
validateOperation(activeDefaultDataverse, stmt);
rewriteStatement(stmt); // Rewrite the statement's AST.
@@ -338,13 +340,13 @@
break;
case Statement.Kind.CREATE_PRIMARY_FEED:
case Statement.Kind.CREATE_SECONDARY_FEED:
- handleCreateFeedStatement(metadataProvider, stmt, hcc);
+ handleCreateFeedStatement(metadataProvider, stmt);
break;
case Statement.Kind.DROP_FEED:
handleDropFeedStatement(metadataProvider, stmt, hcc);
break;
case Statement.Kind.DROP_FEED_POLICY:
- handleDropFeedPolicyStatement(metadataProvider, stmt, hcc);
+ handleDropFeedPolicyStatement(metadataProvider, stmt);
break;
case Statement.Kind.CONNECT_FEED:
handleConnectFeedStatement(metadataProvider, stmt, hcc);
@@ -356,7 +358,7 @@
handleSubscribeFeedStatement(metadataProvider, stmt, hcc);
break;
case Statement.Kind.CREATE_FEED_POLICY:
- handleCreateFeedPolicyStatement(metadataProvider, stmt, hcc);
+ handleCreateFeedPolicyStatement(metadataProvider, stmt);
break;
case Statement.Kind.QUERY:
metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
@@ -378,9 +380,14 @@
case Statement.Kind.RUN:
handleRunStatement(metadataProvider, stmt, hcc);
break;
- default:
- // Default should delegate unknown statement to extension-manager
+ case Statement.Kind.FUNCTION_DECL:
+ // No op
break;
+ case Statement.Kind.EXTENSION:
+ ((IExtensionStatement) stmt).handle(this, metadataProvider, hcc);
+ break;
+ default:
+ throw new AsterixException("Unknown function");
}
}
} finally {
@@ -388,14 +395,14 @@
}
}
- private void handleSetStatement(Statement stmt, Map<String, String> config) {
+ protected void handleSetStatement(Statement stmt, Map<String, String> config) {
SetStatement ss = (SetStatement) stmt;
String pname = ss.getPropName();
String pvalue = ss.getPropValue();
config.put(pname, pvalue);
}
- private Pair<IAWriterFactory, FileSplit> handleWriteStatement(Statement stmt)
+ protected Pair<IAWriterFactory, FileSplit> handleWriteStatement(Statement stmt)
throws InstantiationException, IllegalAccessException, ClassNotFoundException {
WriteStatement ws = (WriteStatement) stmt;
File f = new File(ws.getFileName());
@@ -407,7 +414,7 @@
return new Pair<>(writerFactory, outputFile);
}
- private Dataverse handleUseDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+ protected Dataverse handleUseDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt)
throws Exception {
DataverseDecl dvd = (DataverseDecl) stmt;
String dvName = dvd.getDataverseName().getValue();
@@ -429,7 +436,8 @@
}
}
- private void handleCreateDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
+ protected void handleCreateDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
String dvName = stmtCreateDataverse.getDataverseName().getValue();
@@ -458,7 +466,7 @@
}
}
- private void validateCompactionPolicy(String compactionPolicy, Map<String, String> compactionPolicyProperties,
+ protected void validateCompactionPolicy(String compactionPolicy, Map<String, String> compactionPolicyProperties,
MetadataTransactionContext mdTxnCtx, boolean isExternalDataset) throws AsterixException, Exception {
CompactionPolicy compactionPolicyEntity = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
MetadataConstants.METADATA_DATAVERSE_NAME, compactionPolicy);
@@ -466,8 +474,8 @@
throw new AsterixException("Unknown compaction policy: " + compactionPolicy);
}
String compactionPolicyFactoryClassName = compactionPolicyEntity.getClassName();
- ILSMMergePolicyFactory mergePolicyFactory =
- (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
+ ILSMMergePolicyFactory mergePolicyFactory = (ILSMMergePolicyFactory) Class
+ .forName(compactionPolicyFactoryClassName).newInstance();
if (isExternalDataset && mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
throw new AsterixException("The correlated-prefix merge policy cannot be used with external dataset.");
}
@@ -489,10 +497,9 @@
}
}
- private void handleCreateDatasetStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleCreateDatasetStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws AsterixException, Exception {
-
- ProgressState progress = ProgressState.NO_PROGRESS;
+ MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
DatasetDecl dd = (DatasetDecl) stmt;
String dataverseName = getActiveDataverse(dd.getDataverse());
String datasetName = dd.getName().getValue();
@@ -505,7 +512,7 @@
String nodegroupName = getNodeGroupName(ngNameId, dd, dataverseName);
String compactionPolicy = dd.getCompactionPolicy();
Map<String, String> compactionPolicyProperties = dd.getCompactionPolicyProperties();
- boolean defaultCompactionPolicy = (compactionPolicy == null);
+ boolean defaultCompactionPolicy = compactionPolicy == null;
boolean temp = dd.getDatasetDetailsDecl().isTemp();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -535,8 +542,8 @@
if (dt == null) {
throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
}
- String ngName =
- ngNameId != null ? ngNameId.getValue() : configureNodegroupForDataset(dd, dataverseName, mdTxnCtx);
+ String ngName = ngNameId != null ? ngNameId.getValue()
+ : configureNodegroupForDataset(dd, dataverseName, mdTxnCtx);
if (compactionPolicy == null) {
compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
@@ -560,10 +567,10 @@
}
ARecordType metaRecType = (ARecordType) metaItemType;
- List<List<String>> partitioningExprs =
- ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs();
- List<Integer> keySourceIndicators =
- ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getKeySourceIndicators();
+ List<List<String>> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getPartitioningExprs();
+ List<Integer> keySourceIndicators = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getKeySourceIndicators();
boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
ARecordType aRecordType = (ARecordType) itemType;
List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
@@ -573,14 +580,12 @@
if (filterField != null) {
ValidateUtil.validateFilterField(aRecordType, filterField);
}
- if (compactionPolicy == null) {
- if (filterField != null) {
- // If the dataset has a filter and the user didn't specify a merge
- // policy, then we will pick the
- // correlated-prefix as the default merge policy.
- compactionPolicy = GlobalConfig.DEFAULT_FILTERED_DATASET_COMPACTION_POLICY_NAME;
- compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
- }
+ if (compactionPolicy == null && filterField != null) {
+ // If the dataset has a filter and the user didn't specify a merge
+ // policy, then we will pick the
+ // correlated-prefix as the default merge policy.
+ compactionPolicy = GlobalConfig.DEFAULT_FILTERED_DATASET_COMPACTION_POLICY_NAME;
+ compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
}
datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
@@ -588,7 +593,8 @@
break;
case EXTERNAL:
String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
- Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
+ Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getProperties();
datasetDetails = new ExternalDatasetDetails(adapter, properties, new Date(),
ExternalDatasetTransactionState.COMMIT);
@@ -610,15 +616,15 @@
MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
if (dd.getDatasetType() == DatasetType.INTERNAL) {
- Dataverse dataverse =
- MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dataverseName);
- JobSpecification jobSpec =
- DatasetOperations.createDatasetJobSpec(dataverse, datasetName, metadataProvider);
+ Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
+ dataverseName);
+ JobSpecification jobSpec = DatasetOperations.createDatasetJobSpec(dataverse, datasetName,
+ metadataProvider);
// #. make metadataTxn commit before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+ progress.setValue(ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA);
// #. runJob
JobUtils.runJob(hcc, jobSpec, true);
@@ -639,7 +645,7 @@
abort(e, e, mdTxnCtx);
}
- if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+ if (progress.getValue() == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
// #. execute compensation operations
// remove the index in NC
@@ -686,7 +692,7 @@
}
}
- private void validateIfResourceIsActiveInFeed(String dataverseName, String datasetName) throws AsterixException {
+ protected void validateIfResourceIsActiveInFeed(String dataverseName, String datasetName) throws AsterixException {
StringBuilder builder = null;
IActiveEntityEventsListener[] listeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
for (IActiveEntityEventsListener listener : listeners) {
@@ -704,7 +710,7 @@
}
}
- private String getNodeGroupName(Identifier ngNameId, DatasetDecl dd, String dataverse) {
+ protected String getNodeGroupName(Identifier ngNameId, DatasetDecl dd, String dataverse) {
if (ngNameId != null) {
return ngNameId.getValue();
}
@@ -716,7 +722,8 @@
}
}
- private String configureNodegroupForDataset(DatasetDecl dd, String dataverse, MetadataTransactionContext mdTxnCtx)
+ protected String configureNodegroupForDataset(DatasetDecl dd, String dataverse,
+ MetadataTransactionContext mdTxnCtx)
throws AsterixException {
int nodegroupCardinality;
String nodegroupName;
@@ -734,9 +741,9 @@
nodegroupCardinality = Integer.parseInt(dd.getHints().get(DatasetNodegroupCardinalityHint.NAME));
}
List<String> nodeNames = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames();
- List<String> nodeNamesClone = new ArrayList<String>(nodeNames);
+ List<String> nodeNamesClone = new ArrayList<>(nodeNames);
String metadataNodeName = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
- List<String> selectedNodes = new ArrayList<String>();
+ List<String> selectedNodes = new ArrayList<>();
selectedNodes.add(metadataNodeName);
numChosen++;
nodeNamesClone.remove(metadataNodeName);
@@ -765,7 +772,7 @@
}
- private void handleCreateIndexStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleCreateIndexStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
ProgressState progress = ProgressState.NO_PROGRESS;
CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
@@ -809,23 +816,20 @@
metaRecordType = (ARecordType) metaDt.getDatatype();
}
- List<List<String>> indexFields = new ArrayList<List<String>>();
- List<IAType> indexFieldTypes = new ArrayList<IAType>();
+ List<List<String>> indexFields = new ArrayList<>();
+ List<IAType> indexFieldTypes = new ArrayList<>();
int keyIndex = 0;
for (Pair<List<String>, TypeExpression> fieldExpr : stmtCreateIndex.getFieldExprs()) {
IAType fieldType = null;
- ARecordType subType =
- KeyFieldTypeUtils.chooseSource(keySourceIndicators, keyIndex, aRecordType, metaRecordType);
+ ARecordType subType = KeyFieldTypeUtils.chooseSource(keySourceIndicators, keyIndex, aRecordType,
+ metaRecordType);
boolean isOpen = subType.isOpen();
int i = 0;
if (fieldExpr.first.size() > 1 && !isOpen) {
- for (; i < fieldExpr.first.size() - 1;) {
+ while (i < fieldExpr.first.size() - 1 && !isOpen) {
subType = (ARecordType) subType.getFieldType(fieldExpr.first.get(i));
i++;
- if (subType.isOpen()) {
- isOpen = true;
- break;
- } ;
+ isOpen = subType.isOpen();
}
}
if (fieldExpr.second == null) {
@@ -842,8 +846,8 @@
if (stmtCreateIndex.hasMetaField()) {
throw new AlgebricksException("Typed open index can only be created on the record part");
}
- Map<TypeSignature, IAType> typeMap =
- TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second, indexName, dataverseName);
+ Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second,
+ indexName, dataverseName);
TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
fieldType = typeMap.get(typeSignature);
}
@@ -910,7 +914,7 @@
// Check if the files index exist
filesIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
- firstExternalDatasetIndex = (filesIndex == null);
+ firstExternalDatasetIndex = filesIndex == null;
// Lock external dataset
ExternalDatasetsRegistry.INSTANCE.buildIndexBegin(ds, firstExternalDatasetIndex);
datasetLocked = true;
@@ -1044,8 +1048,8 @@
CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
ExternalIndexingOperations.getFilesIndexName(datasetName));
try {
- JobSpecification jobSpec =
- ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds, metadataProvider, ds);
+ JobSpecification jobSpec = ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds,
+ metadataProvider, ds);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
JobUtils.runJob(hcc, jobSpec, true);
@@ -1065,8 +1069,8 @@
metadataProvider.setMetadataTxnContext(mdTxnCtx);
CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
try {
- JobSpecification jobSpec =
- IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds);
+ JobSpecification jobSpec = IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
+ ds);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
@@ -1129,7 +1133,7 @@
}
}
- private void handleCreateTypeStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
+ protected void handleCreateTypeStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
TypeDecl stmtCreateType = (TypeDecl) stmt;
String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
String typeName = stmtCreateType.getIdent().getValue();
@@ -1167,7 +1171,7 @@
}
}
- private void handleDataverseDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleDataverseDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
String dataverseName = stmtDelete.getDataverseName().getValue();
@@ -1177,7 +1181,7 @@
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
MetadataLockManager.INSTANCE.acquireDataverseWriteLock(dataverseName);
- List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+ List<JobSpecification> jobsToExecute = new ArrayList<>();
try {
Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
if (dv == null) {
@@ -1212,8 +1216,8 @@
String datasetName = datasets.get(j).getDatasetName();
DatasetType dsType = datasets.get(j).getDatasetType();
if (dsType == DatasetType.INTERNAL) {
- List<Index> indexes =
- MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+ datasetName);
for (int k = 0; k < indexes.size(); k++) {
if (indexes.get(k).isSecondaryIndex()) {
CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
@@ -1227,8 +1231,8 @@
jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
} else {
// External dataset
- List<Index> indexes =
- MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+ datasetName);
for (int k = 0; k < indexes.size(); k++) {
if (ExternalIndexingOperations.isFileIndex(indexes.get(k))) {
CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
@@ -1312,7 +1316,7 @@
}
}
- private void disconnectFeedBeforeDelete(Identifier dvId, EntityId activeEntityId, FeedConnectionId conn,
+ protected void disconnectFeedBeforeDelete(Identifier dvId, EntityId activeEntityId, FeedConnectionId conn,
AqlMetadataProvider metadataProvider, IHyracksClientConnection hcc) {
DisconnectFeedStatement disStmt = new DisconnectFeedStatement(dvId,
new Identifier(activeEntityId.getEntityName()), new Identifier(conn.getDatasetName()));
@@ -1330,150 +1334,44 @@
}
}
- private void handleDatasetDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected Dataset getDataset(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName)
+ throws MetadataException {
+ return MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+ }
+
+ protected void handleDatasetDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
- DropStatement stmtDelete = (DropStatement) stmt;
+ DropDatasetStatement stmtDelete = (DropDatasetStatement) stmt;
String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
String datasetName = stmtDelete.getDatasetName().getValue();
-
- ProgressState progress = ProgressState.NO_PROGRESS;
- MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- boolean bActiveTxn = true;
- metadataProvider.setMetadataTxnContext(mdTxnCtx);
-
+ MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
+ MutableObject<MetadataTransactionContext> mdTxnCtx = new MutableObject<>(
+ MetadataManager.INSTANCE.beginTransaction());
+ MutableBoolean bActiveTxn = new MutableBoolean(true);
+ metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
MetadataLockManager.INSTANCE.dropDatasetBegin(dataverseName, dataverseName + "." + datasetName);
- List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+ List<JobSpecification> jobsToExecute = new ArrayList<>();
try {
-
- Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+ Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
if (ds == null) {
if (stmtDelete.getIfExists()) {
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
return;
} else {
- throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
- + dataverseName + ".");
+ throw new AlgebricksException("There is no dataset with this name " + datasetName
+ + " in dataverse " + dataverseName + ".");
}
}
- Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<>();
- if (ds.getDatasetType() == DatasetType.INTERNAL) {
- // prepare job spec(s) that would disconnect any active feeds involving the dataset.
- IActiveEntityEventsListener[] feedConnections =
- ActiveJobNotificationHandler.INSTANCE.getEventListeners();
- for (IActiveEntityEventsListener conn : feedConnections) {
- if (conn.getEntityId().getExtensionName().equals(Feed.EXTENSION_NAME)
- && ((FeedEventsListener) conn).isConnectedToDataset(datasetName)) {
- FeedConnectionId connectionId = new FeedConnectionId(conn.getEntityId(), datasetName);
- Pair<JobSpecification, Boolean> p =
- FeedOperations.buildDisconnectFeedJobSpec(metadataProvider, connectionId);
- disconnectJobList.put(connectionId, p);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Disconnecting feed " + connectionId.getFeedId().getEntityName()
- + " from dataset " + datasetName + " as dataset is being dropped");
- }
- // prepare job to remove feed log storage
- jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(
- MetadataManager.INSTANCE.getFeed(mdTxnCtx, connectionId.getFeedId().getDataverse(),
- connectionId.getFeedId().getEntityName())));
- }
- }
-
- // #. prepare jobs to drop the datatset and the indexes in NC
- List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
- for (int j = 0; j < indexes.size(); j++) {
- if (indexes.get(j).isSecondaryIndex()) {
- CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
- indexes.get(j).getIndexName());
- jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
- }
- }
- CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
- jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
-
- // #. mark the existing dataset as PendingDropOp
- MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
- MetadataManager.INSTANCE.addDataset(mdTxnCtx,
- new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
- ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName(), ds.getNodeGroupName(),
- ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(), ds.getDatasetDetails(),
- ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
- IMetadataEntity.PENDING_DROP_OP));
-
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- bActiveTxn = false;
- progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
-
- // # disconnect the feeds
- for (Pair<JobSpecification, Boolean> p : disconnectJobList.values()) {
- JobUtils.runJob(hcc, p.first, true);
- }
-
- // #. run the jobs
- for (JobSpecification jobSpec : jobsToExecute) {
- JobUtils.runJob(hcc, jobSpec, true);
- }
-
- mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- bActiveTxn = true;
- metadataProvider.setMetadataTxnContext(mdTxnCtx);
- } else {
- // External dataset
- ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
- // #. prepare jobs to drop the datatset and the indexes in NC
- List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
- for (int j = 0; j < indexes.size(); j++) {
- if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
- CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
- indexes.get(j).getIndexName());
- jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
- } else {
- CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
- indexes.get(j).getIndexName());
- jobsToExecute
- .add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds, metadataProvider, ds));
- }
- }
-
- // #. mark the existing dataset as PendingDropOp
- MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
- MetadataManager.INSTANCE.addDataset(mdTxnCtx,
- new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
- ds.getNodeGroupName(), ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(),
- ds.getDatasetDetails(), ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
- IMetadataEntity.PENDING_DROP_OP));
-
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- bActiveTxn = false;
- progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
-
- // #. run the jobs
- for (JobSpecification jobSpec : jobsToExecute) {
- JobUtils.runJob(hcc, jobSpec, true);
- }
- if (indexes.size() > 0) {
- ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
- }
- mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- bActiveTxn = true;
- metadataProvider.setMetadataTxnContext(mdTxnCtx);
- }
-
- // #. finally, delete the dataset.
- MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
- // Drop the associated nodegroup
- String nodegroup = ds.getNodeGroupName();
- if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
- MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, dataverseName + ":" + datasetName);
- }
-
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ doDropDataset(ds, datasetName, metadataProvider, mdTxnCtx, jobsToExecute, dataverseName, bActiveTxn,
+ progress, hcc);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
} catch (Exception e) {
- if (bActiveTxn) {
- abort(e, e, mdTxnCtx);
+ if (bActiveTxn.booleanValue()) {
+ abort(e, e, mdTxnCtx.getValue());
}
- if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+ if (progress.getValue() == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
// #. execute compensation operations
// remove the all indexes in NC
try {
@@ -1486,15 +1384,15 @@
}
// remove the record from the metadata.
- mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
+ metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
try {
MetadataManager.INSTANCE.dropDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
datasetName);
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
} catch (Exception e2) {
e.addSuppressed(e2);
- abort(e, e2, mdTxnCtx);
+ abort(e, e2, mdTxnCtx.getValue());
throw new IllegalStateException("System is inconsistent state: pending dataset(" + dataverseName
+ "." + datasetName + ") couldn't be removed from the metadata", e);
}
@@ -1506,7 +1404,123 @@
}
}
- private void handleIndexDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void doDropDataset(Dataset ds, String datasetName, AqlMetadataProvider metadataProvider,
+ MutableObject<MetadataTransactionContext> mdTxnCtx, List<JobSpecification> jobsToExecute,
+ String dataverseName, MutableBoolean bActiveTxn, MutableObject<ProgressState> progress,
+ IHyracksClientConnection hcc) throws Exception {
+ Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<>();
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
+ // prepare job spec(s) that would disconnect any active feeds involving the dataset.
+ IActiveEntityEventsListener[] feedConnections = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
+ for (IActiveEntityEventsListener conn : feedConnections) {
+ if (conn.getEntityId().getExtensionName().equals(Feed.EXTENSION_NAME)
+ && ((FeedEventsListener) conn).isConnectedToDataset(datasetName)) {
+ FeedConnectionId connectionId = new FeedConnectionId(conn.getEntityId(), datasetName);
+ Pair<JobSpecification, Boolean> p = FeedOperations.buildDisconnectFeedJobSpec(metadataProvider,
+ connectionId);
+ disconnectJobList.put(connectionId, p);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disconnecting feed " + connectionId.getFeedId().getEntityName() + " from dataset "
+ + datasetName + " as dataset is being dropped");
+ }
+ // prepare job to remove feed log storage
+ jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(MetadataManager.INSTANCE.getFeed(
+ mdTxnCtx.getValue(), connectionId.getFeedId().getDataverse(),
+ connectionId.getFeedId().getEntityName())));
+ }
+ }
+
+ // #. prepare jobs to drop the datatset and the indexes in NC
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName,
+ datasetName);
+ for (int j = 0; j < indexes.size(); j++) {
+ if (indexes.get(j).isSecondaryIndex()) {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(j).getIndexName());
+ jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+ }
+ }
+ CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
+ jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
+
+ // #. mark the existing dataset as PendingDropOp
+ MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
+ MetadataManager.INSTANCE.addDataset(mdTxnCtx.getValue(),
+ new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
+ ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName(), ds.getNodeGroupName(),
+ ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(), ds.getDatasetDetails(),
+ ds.getHints(), ds.getDatasetType(), ds.getDatasetId(), IMetadataEntity.PENDING_DROP_OP));
+
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
+ bActiveTxn.setValue(false);
+ progress.setValue(ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA);
+
+ // # disconnect the feeds
+ for (Pair<JobSpecification, Boolean> p : disconnectJobList.values()) {
+ JobUtils.runJob(hcc, p.first, true);
+ }
+
+ // #. run the jobs
+ for (JobSpecification jobSpec : jobsToExecute) {
+ JobUtils.runJob(hcc, jobSpec, true);
+ }
+
+ mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
+ bActiveTxn.setValue(true);
+ metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
+ } else {
+ // External dataset
+ ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
+ // #. prepare jobs to drop the datatset and the indexes in NC
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName,
+ datasetName);
+ for (int j = 0; j < indexes.size(); j++) {
+ if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(j).getIndexName());
+ jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+ } else {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(j).getIndexName());
+ jobsToExecute.add(
+ ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds, metadataProvider, ds));
+ }
+ }
+
+ // #. mark the existing dataset as PendingDropOp
+ MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
+ MetadataManager.INSTANCE.addDataset(mdTxnCtx.getValue(),
+ new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
+ ds.getNodeGroupName(), ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(),
+ ds.getDatasetDetails(), ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
+ IMetadataEntity.PENDING_DROP_OP));
+
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
+ bActiveTxn.setValue(false);
+ progress.setValue(ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA);
+
+ // #. run the jobs
+ for (JobSpecification jobSpec : jobsToExecute) {
+ JobUtils.runJob(hcc, jobSpec, true);
+ }
+ if (!indexes.isEmpty()) {
+ ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
+ }
+ mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
+ bActiveTxn.setValue(true);
+ metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
+ }
+
+ // #. finally, delete the dataset.
+ MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
+ // Drop the associated nodegroup
+ String nodegroup = ds.getNodeGroupName();
+ if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
+ MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx.getValue(), dataverseName + ":" + datasetName);
+ }
+ }
+
+ protected void handleIndexDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
@@ -1522,7 +1536,7 @@
String indexName = null;
// For external index
boolean dropFilesIndex = false;
- List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+ List<JobSpecification> jobsToExecute = new ArrayList<>();
try {
Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
@@ -1565,9 +1579,11 @@
// #. mark PendingDropOp on the existing index
MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
MetadataManager.INSTANCE.addIndex(mdTxnCtx,
- new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
+ new Index(dataverseName, datasetName, indexName, index.getIndexType(),
+ index.getKeyFieldNames(),
index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
- index.isEnforcingKeyFileds(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+ index.isEnforcingKeyFileds(), index.isPrimaryIndex(),
+ IMetadataEntity.PENDING_DROP_OP));
// #. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1602,8 +1618,8 @@
// #. prepare a job to drop the index in NC.
CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
- List<Index> datasetIndexes =
- MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+ datasetName);
if (datasetIndexes.size() == 2) {
dropFilesIndex = true;
// only one index + the files index, we need to delete both of the indexes
@@ -1629,9 +1645,11 @@
// #. mark PendingDropOp on the existing index
MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
MetadataManager.INSTANCE.addIndex(mdTxnCtx,
- new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
+ new Index(dataverseName, datasetName, indexName, index.getIndexType(),
+ index.getKeyFieldNames(),
index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
- index.isEnforcingKeyFileds(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+ index.isEnforcingKeyFileds(), index.isPrimaryIndex(),
+ IMetadataEntity.PENDING_DROP_OP));
// #. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1690,8 +1708,9 @@
} catch (Exception e2) {
e.addSuppressed(e2);
abort(e, e2, mdTxnCtx);
- throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName + "."
- + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
+ throw new IllegalStateException("System is inconsistent state: pending index("
+ + dataverseName + "." + datasetName + "."
+ + indexName + ") couldn't be removed from the metadata", e);
}
}
@@ -1702,7 +1721,7 @@
}
}
- private void handleTypeDropStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
+ protected void handleTypeDropStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
String dataverseName = getActiveDataverse(stmtTypeDrop.getDataverseName());
@@ -1730,8 +1749,8 @@
}
}
- private void handleNodegroupDropStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
-
+ protected void handleNodegroupDropStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
String nodegroupName = stmtDelete.getNodeGroupName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1756,7 +1775,8 @@
}
}
- private void handleCreateFunctionStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
+ protected void handleCreateFunctionStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
String dataverse = getActiveDataverseName(cfs.getSignature().getNamespace());
String functionName = cfs.getaAterixFunction().getName();
@@ -1784,7 +1804,7 @@
}
}
- private void handleFunctionDropStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
+ protected void handleFunctionDropStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
FunctionSignature signature = stmtDropFunction.getFunctionSignature();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1810,8 +1830,8 @@
}
}
- private void handleLoadStatement(AqlMetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
- throws Exception {
+ protected void handleLoadStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
LoadStatement loadStmt = (LoadStatement) stmt;
String dataverseName = getActiveDataverse(loadStmt.getDataverseName());
String datasetName = loadStmt.getDatasetName().getValue();
@@ -1820,11 +1840,11 @@
metadataProvider.setMetadataTxnContext(mdTxnCtx);
MetadataLockManager.INSTANCE.modifyDatasetBegin(dataverseName, dataverseName + "." + datasetName);
try {
- CompiledLoadFromFileStatement cls =
- new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
- loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
- JobSpecification spec =
- apiFramework.compileQuery(null, metadataProvider, null, 0, null, sessionConfig, cls);
+ CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName,
+ loadStmt.getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
+ loadStmt.dataIsAlreadySorted());
+ JobSpecification spec = apiFramework.compileQuery(null, metadataProvider, null, 0, null, sessionConfig,
+ cls);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
if (spec != null) {
@@ -1840,7 +1860,7 @@
}
}
- private void handleInsertUpsertStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleInsertUpsertStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
@@ -1888,7 +1908,7 @@
}
}
- private void handleDeleteStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleDeleteStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
DeleteStatement stmtDelete = (DeleteStatement) stmt;
@@ -1926,13 +1946,14 @@
}
}
- private JobSpecification rewriteCompileQuery(AqlMetadataProvider metadataProvider, Query query,
+ @Override
+ public JobSpecification rewriteCompileQuery(AqlMetadataProvider metadataProvider, Query query,
ICompiledDmlStatement stmt)
throws AsterixException, RemoteException, AlgebricksException, JSONException, ACIDException {
// Query Rewriting (happens under the same ongoing metadata transaction)
- Pair<Query, Integer> reWrittenQuery =
- apiFramework.reWriteQuery(declaredFunctions, metadataProvider, query, sessionConfig);
+ Pair<Query, Integer> reWrittenQuery = apiFramework.reWriteQuery(declaredFunctions, metadataProvider, query,
+ sessionConfig);
// Query Compilation (happens under the same ongoing metadata transaction)
JobSpecification spec = apiFramework.compileQuery(declaredFunctions, metadataProvider, reWrittenQuery.first,
@@ -1942,8 +1963,7 @@
}
- private void handleCreateFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
- IHyracksClientConnection hcc) throws Exception {
+ protected void handleCreateFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
CreateFeedStatement cfs = (CreateFeedStatement) stmt;
String dataverseName = getActiveDataverse(cfs.getDataverseName());
String feedName = cfs.getFeedName().getValue();
@@ -1988,8 +2008,7 @@
}
}
- private void handleCreateFeedPolicyStatement(AqlMetadataProvider metadataProvider, Statement stmt,
- IHyracksClientConnection hcc) throws Exception {
+ protected void handleCreateFeedPolicyStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
String dataverse;
@@ -2000,8 +2019,8 @@
policy = cfps.getPolicyName();
MetadataLockManager.INSTANCE.createFeedPolicyBegin(dataverse, dataverse + "." + policy);
try {
- FeedPolicyEntity feedPolicy =
- MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, policy);
+ FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE
+ .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, policy);
if (feedPolicy != null) {
if (cfps.getIfNotExists()) {
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2014,7 +2033,8 @@
String description = cfps.getDescription() == null ? "" : cfps.getDescription();
if (extendingExisting) {
FeedPolicyEntity sourceFeedPolicy = MetadataManager.INSTANCE
- .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, cfps.getSourcePolicyName());
+ .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse,
+ cfps.getSourcePolicyName());
if (sourceFeedPolicy == null) {
sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
MetadataConstants.METADATA_DATAVERSE_NAME, cfps.getSourcePolicyName());
@@ -2031,9 +2051,9 @@
InputStream stream = new FileInputStream(cfps.getSourcePolicyFile());
prop.load(stream);
} catch (Exception e) {
- throw new AlgebricksException("Unable to read policy file" + cfps.getSourcePolicyFile());
+ throw new AlgebricksException("Unable to read policy file" + cfps.getSourcePolicyFile(), e);
}
- Map<String, String> policyProperties = new HashMap<String, String>();
+ Map<String, String> policyProperties = new HashMap<>();
for (Entry<Object, Object> entry : prop.entrySet()) {
policyProperties.put((String) entry.getKey(), (String) entry.getValue());
}
@@ -2049,7 +2069,7 @@
}
}
- private void handleDropFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleDropFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
String dataverseName = getActiveDataverse(stmtFeedDrop.getDataverseName());
@@ -2069,8 +2089,8 @@
}
EntityId feedId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
- FeedEventsListener listener =
- (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(feedId);
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(feedId);
if (listener != null) {
StringBuilder builder = new StringBuilder();
for (FeedConnectionId connectionId : listener.getConnections()) {
@@ -2098,9 +2118,8 @@
}
}
- private void handleDropFeedPolicyStatement(AqlMetadataProvider metadataProvider, Statement stmt,
- IHyracksClientConnection hcc) throws Exception {
-
+ protected void handleDropFeedPolicyStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
@@ -2127,9 +2146,8 @@
}
}
- private void handleConnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleConnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
-
ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
String dataverseName = getActiveDataverse(cfs.getDataverseName());
String feedName = cfs.getFeedName();
@@ -2141,8 +2159,8 @@
IFeedLifecycleEventSubscriber eventSubscriber = new FeedLifecycleEventSubscriber();
FeedConnectionId feedConnId = null;
EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, cfs.getFeedName());
- FeedEventsListener listener =
- (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(entityId);
MetadataLockManager.INSTANCE.connectFeedBegin(dataverseName, dataverseName + "." + datasetName,
dataverseName + "." + feedName);
try {
@@ -2161,26 +2179,26 @@
throw new AsterixException("Feed " + cfs.getFeedName() + " is already connected to dataset "
+ cfs.getDatasetName().getValue());
}
- FeedPolicyEntity feedPolicy =
- FeedMetadataUtil.validateIfPolicyExists(dataverseName, cbfs.getPolicyName(), mdTxnCtx);
+ FeedPolicyEntity feedPolicy = FeedMetadataUtil.validateIfPolicyExists(dataverseName, cbfs.getPolicyName(),
+ mdTxnCtx);
// All Metadata checks have passed. Feed connect request is valid. //
if (listener == null) {
listener = new FeedEventsListener(entityId);
ActiveJobNotificationHandler.INSTANCE.registerListener(listener);
}
FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(feedPolicy.getProperties());
- Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> triple =
- getFeedConnectionRequest(dataverseName, feed, cbfs.getDatasetName(), feedPolicy, mdTxnCtx);
+ Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> triple = getFeedConnectionRequest(dataverseName,
+ feed, cbfs.getDatasetName(), feedPolicy, mdTxnCtx);
FeedConnectionRequest connectionRequest = triple.first;
boolean createFeedIntakeJob = triple.second;
listener.registerFeedEventSubscriber(eventSubscriber);
subscriberRegistered = true;
if (createFeedIntakeJob) {
EntityId feedId = connectionRequest.getFeedJointKey().getFeedId();
- Feed primaryFeed =
- MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName());
- Pair<JobSpecification, IAdapterFactory> pair =
- FeedOperations.buildFeedIntakeJobSpec(primaryFeed, metadataProvider, policyAccessor);
+ Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(),
+ feedId.getEntityName());
+ Pair<JobSpecification, IAdapterFactory> pair = FeedOperations.buildFeedIntakeJobSpec(primaryFeed,
+ metadataProvider, policyAccessor);
// adapter configuration are valid at this stage
// register the feed joints (these are auto-de-registered)
int numOfPrividers = pair.second.getPartitionConstraint().getLocations().length;
@@ -2191,14 +2209,6 @@
triple.third.get(0), pair.first);
pair.first.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, activeJob);
JobUtils.runJob(hcc, pair.first, false);
- /*
- * TODO: Fix record tracking
- * IFeedAdapterFactory adapterFactory = pair.second;
- * if (adapterFactory.isRecordTrackingEnabled()) {
- * FeedLifecycleListener.INSTANCE.registerFeedIntakeProgressTracker(feedConnId,
- * adapterFactory.createIntakeProgressTracker());
- * }
- */
eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_INTAKE_STARTED);
} else {
for (IFeedJoint fj : triple.third) {
@@ -2237,20 +2247,20 @@
* @return
* @throws AsterixException
*/
- private Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> getFeedConnectionRequest(String dataverse,
+ protected Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> getFeedConnectionRequest(String dataverse,
Feed feed, String dataset, FeedPolicyEntity feedPolicy, MetadataTransactionContext mdTxnCtx)
throws AsterixException {
- IFeedJoint sourceFeedJoint = null;
- FeedConnectionRequest request = null;
- List<String> functionsToApply = new ArrayList<String>();
+ IFeedJoint sourceFeedJoint;
+ FeedConnectionRequest request;
+ List<String> functionsToApply = new ArrayList<>();
boolean needIntakeJob = false;
- List<IFeedJoint> jointsToRegister = new ArrayList<IFeedJoint>();
+ List<IFeedJoint> jointsToRegister = new ArrayList<>();
FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), dataset);
- FeedRuntimeType connectionLocation = null;
+ FeedRuntimeType connectionLocation;
FeedJointKey feedJointKey = getFeedJointKey(feed, mdTxnCtx);
EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverse, feed.getFeedName());
- FeedEventsListener listener =
- (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(entityId);
if (listener == null) {
throw new AsterixException("Feed Listener is not registered");
}
@@ -2262,7 +2272,7 @@
connectionLocation = FeedRuntimeType.INTAKE;
EntityId sourceFeedId = feedJointKey.getFeedId(); // the root/primary feedId
Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, sourceFeedId.getEntityName());
- FeedJointKey intakeFeedJointKey = new FeedJointKey(sourceFeedId, new ArrayList<String>());
+ FeedJointKey intakeFeedJointKey = new FeedJointKey(sourceFeedId, new ArrayList<>());
sourceFeedJoint = new FeedJoint(intakeFeedJointKey, primaryFeed.getFeedId(), connectionLocation,
FeedJointType.INTAKE, connectionId);
jointsToRegister.add(sourceFeedJoint);
@@ -2298,22 +2308,22 @@
dataset, feedPolicy.getPolicyName(), feedPolicy.getProperties(), feed.getFeedId());
sourceFeedJoint.addConnectionRequest(request);
- return new Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>>(request, needIntakeJob, jointsToRegister);
+ return new Triple<>(request, needIntakeJob, jointsToRegister);
}
/*
* Gets the feed joint corresponding to the feed definition. Tuples constituting the feed are
* available at this feed joint.
*/
- private FeedJointKey getFeedJointKey(Feed feed, MetadataTransactionContext ctx) throws MetadataException {
+ protected FeedJointKey getFeedJointKey(Feed feed, MetadataTransactionContext ctx) throws MetadataException {
Feed sourceFeed = feed;
- List<String> appliedFunctions = new ArrayList<String>();
+ List<String> appliedFunctions = new ArrayList<>();
while (sourceFeed.getFeedType().equals(IFeed.FeedType.SECONDARY)) {
if (sourceFeed.getAppliedFunction() != null) {
appliedFunctions.add(0, sourceFeed.getAppliedFunction().getName());
}
- Feed parentFeed =
- MetadataManager.INSTANCE.getFeed(ctx, feed.getDataverseName(), sourceFeed.getSourceFeedName());
+ Feed parentFeed = MetadataManager.INSTANCE.getFeed(ctx, feed.getDataverseName(),
+ sourceFeed.getSourceFeedName());
sourceFeed = parentFeed;
}
@@ -2324,7 +2334,7 @@
return new FeedJointKey(sourceFeed.getFeedId(), appliedFunctions);
}
- private void handleDisconnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleDisconnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
String dataverseName = getActiveDataverse(cfs.getDataverseName());
@@ -2338,8 +2348,8 @@
EntityId entityId = new EntityId(Feed.EXTENSION_NAME, feed.getDataverseName(), feed.getFeedName());
FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), cfs.getDatasetName().getValue());
IFeedLifecycleEventSubscriber eventSubscriber = new FeedLifecycleEventSubscriber();
- FeedEventsListener listener =
- (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(entityId);
if (listener == null || !listener.isConnectedToDataset(datasetName)) {
throw new AsterixException("Feed " + feed.getFeedId().getEntityName() + " is currently not connected to "
+ cfs.getDatasetName().getValue() + ". Invalid operation!");
@@ -2354,8 +2364,8 @@
throw new AsterixException(
"Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse " + dataverseName);
}
- Pair<JobSpecification, Boolean> specDisconnectType =
- FeedOperations.buildDisconnectFeedJobSpec(metadataProvider, connectionId);
+ Pair<JobSpecification, Boolean> specDisconnectType = FeedOperations
+ .buildDisconnectFeedJobSpec(metadataProvider, connectionId);
JobSpecification jobSpec = specDisconnectType.first;
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
@@ -2372,7 +2382,7 @@
}
}
- private void handleSubscribeFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleSubscribeFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
@@ -2386,8 +2396,8 @@
SubscribeFeedStatement bfs = (SubscribeFeedStatement) stmt;
bfs.initialize(metadataProvider.getMetadataTxnContext());
- CompiledSubscribeFeedStatement csfs =
- new CompiledSubscribeFeedStatement(bfs.getSubscriptionRequest(), bfs.getVarCounter());
+ CompiledSubscribeFeedStatement csfs = new CompiledSubscribeFeedStatement(bfs.getSubscriptionRequest(),
+ bfs.getVarCounter());
metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, "" + bfs.getPolicy());
metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
@@ -2416,8 +2426,8 @@
if (compiled != null) {
FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
.getActiveEntityListener(bfs.getSubscriptionRequest().getReceivingFeedId());
- FeedConnectJobInfo activeJob = new FeedConnectJobInfo(bfs.getSubscriptionRequest().getReceivingFeedId(),
- null, ActivityState.ACTIVE,
+ FeedConnectJobInfo activeJob = new FeedConnectJobInfo(
+ bfs.getSubscriptionRequest().getReceivingFeedId(), null, ActivityState.ACTIVE,
new FeedConnectionId(bfs.getSubscriptionRequest().getReceivingFeedId(), dataset),
listener.getSourceFeedJoint(), null, alteredJobSpec, policy.getProperties());
alteredJobSpec.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, activeJob);
@@ -2425,7 +2435,7 @@
}
} catch (Exception e) {
- e.printStackTrace();
+ LOGGER.log(Level.WARNING, e.getMessage(), e);
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
}
@@ -2436,7 +2446,7 @@
}
}
- private void handleCompactStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleCompactStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
CompactStatement compactStatement = (CompactStatement) stmt;
String dataverseName = getActiveDataverse(compactStatement.getDataverseName());
@@ -2445,7 +2455,7 @@
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
MetadataLockManager.INSTANCE.compactBegin(dataverseName, dataverseName + "." + datasetName);
- List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+ List<JobSpecification> jobsToExecute = new ArrayList<>();
try {
Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
if (ds == null) {
@@ -2457,26 +2467,29 @@
ds.getItemTypeDataverseName(), itemTypeName);
ARecordType metaRecordType = null;
if (ds.hasMetaPart()) {
- metaRecordType =
- (ARecordType) MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
- ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName()).getDatatype();
+ metaRecordType = (ARecordType) MetadataManager.INSTANCE
+ .getDatatype(metadataProvider.getMetadataTxnContext(), ds.getMetaItemTypeDataverseName(),
+ ds.getMetaItemTypeName())
+ .getDatatype();
}
// Prepare jobs to compact the datatset and its indexes
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
- if (indexes.size() == 0) {
+ if (indexes.isEmpty()) {
throw new AlgebricksException(
"Cannot compact the extrenal dataset " + datasetName + " because it has no indexes");
}
- Dataverse dataverse =
- MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dataverseName);
+ Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
+ dataverseName);
jobsToExecute.add(DatasetOperations.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
ARecordType aRecordType = (ARecordType) dt.getDatatype();
- ARecordType enforcedType = IntroduceSecondaryIndexInsertDeleteRule.createEnforcedType(aRecordType, indexes);
+ ARecordType enforcedType = IntroduceSecondaryIndexInsertDeleteRule.createEnforcedType(
+ aRecordType, indexes);
if (ds.getDatasetType() == DatasetType.INTERNAL) {
for (int j = 0; j < indexes.size(); j++) {
if (indexes.get(j).isSecondaryIndex()) {
jobsToExecute
- .add(DatasetOperations.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
+ .add(DatasetOperations.compactDatasetJobSpec(
+ dataverse, datasetName, metadataProvider));
}
}
} else {
@@ -2500,10 +2513,9 @@
}
}
- private void prepareCompactJobsForExternalDataset(List<Index> indexes, String dataverseName, String datasetName,
+ protected void prepareCompactJobsForExternalDataset(List<Index> indexes, String dataverseName, String datasetName,
Dataset ds, List<JobSpecification> jobsToExecute, ARecordType aRecordType, ARecordType metaRecordType,
- AqlMetadataProvider metadataProvider, ARecordType enforcedType)
- throws MetadataException, AlgebricksException {
+ AqlMetadataProvider metadataProvider, ARecordType enforcedType) throws AlgebricksException {
for (int j = 0; j < indexes.size(); j++) {
if (!ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
CompiledIndexCompactStatement cics = new CompiledIndexCompactStatement(dataverseName, datasetName,
@@ -2522,8 +2534,8 @@
jobsToExecute.add(ExternalIndexingOperations.compactFilesIndexJobSpec(ds, metadataProvider));
}
- private void handleQuery(AqlMetadataProvider metadataProvider, Query query, IHyracksClientConnection hcc,
- IHyracksDataset hdc, ResultDelivery resultDelivery, ResultUtils.Stats stats) throws Exception {
+ protected void handleQuery(AqlMetadataProvider metadataProvider, Query query, IHyracksClientConnection hcc,
+ IHyracksDataset hdc, ResultDelivery resultDelivery, Stats stats) throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -2555,9 +2567,9 @@
break;
case SYNC:
hcc.waitForCompletion(jobId);
- ResultReader resultReader = new ResultReader(hcc, hdc);
+ ResultReader resultReader = new ResultReader(hdc);
resultReader.open(jobId, metadataProvider.getResultSetId());
- ResultUtils.displayResults(resultReader, sessionConfig, stats,
+ ResultUtil.displayResults(resultReader, sessionConfig, stats,
metadataProvider.findOutputRecordType());
break;
case ASYNC_DEFERRED:
@@ -2575,7 +2587,7 @@
}
}
} catch (Exception e) {
- e.printStackTrace();
+ LOGGER.log(Level.INFO, e.getMessage(), e);
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
}
@@ -2587,7 +2599,8 @@
}
}
- private void handleCreateNodeGroupStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
+ protected void handleCreateNodeGroupStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
@@ -2604,7 +2617,7 @@
}
} else {
List<Identifier> ncIdentifiers = stmtCreateNodegroup.getNodeControllerNames();
- List<String> ncNames = new ArrayList<String>(ncIdentifiers.size());
+ List<String> ncNames = new ArrayList<>(ncIdentifiers.size());
for (Identifier id : ncIdentifiers) {
ncNames.add(id.getValue());
}
@@ -2619,7 +2632,7 @@
}
}
- private void handleExternalDatasetRefreshStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handleExternalDatasetRefreshStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
String dataverseName = getActiveDataverse(stmtRefresh.getDataverseName());
@@ -2655,7 +2668,7 @@
}
// Dataset has indexes ?
indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
- if (indexes.size() == 0) {
+ if (indexes.isEmpty()) {
throw new AlgebricksException("External dataset " + datasetName + " in dataverse " + dataverseName
+ " doesn't have any index");
}
@@ -2669,9 +2682,9 @@
// Get internal files
metadataFiles = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, ds);
- deletedFiles = new ArrayList<ExternalFile>();
- addedFiles = new ArrayList<ExternalFile>();
- appendedFiles = new ArrayList<ExternalFile>();
+ deletedFiles = new ArrayList<>();
+ addedFiles = new ArrayList<>();
+ appendedFiles = new ArrayList<>();
// Compute delta
// Now we compare snapshot with external file system
@@ -2844,8 +2857,8 @@
}
}
- private void handleRunStatement(AqlMetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
- throws AsterixException, Exception {
+ protected void handleRunStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws AsterixException, Exception {
RunStatement runStmt = (RunStatement) stmt;
switch (runStmt.getSystem()) {
case "pregel":
@@ -2854,12 +2867,13 @@
break;
default:
throw new AlgebricksException(
- "The system \"" + runStmt.getSystem() + "\" specified in your run statement is not supported.");
+ "The system \"" + runStmt.getSystem() +
+ "\" specified in your run statement is not supported.");
}
}
- private void handlePregelixStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ protected void handlePregelixStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
RunStatement pregelixStmt = (RunStatement) stmt;
boolean bActiveTxn = true;
@@ -2870,9 +2884,9 @@
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
- List<String> readDataverses = new ArrayList<String>();
+ List<String> readDataverses = new ArrayList<>();
readDataverses.add(dataverseNameFrom);
- List<String> readDatasets = new ArrayList<String>();
+ List<String> readDatasets = new ArrayList<>();
readDatasets.add(datasetNameFrom);
MetadataLockManager.INSTANCE.insertDeleteUpsertBegin(dataverseNameTo, datasetNameTo, readDataverses,
readDatasets);
@@ -2908,9 +2922,11 @@
// Checks the return state of the external Pregelix command.
if (resultState != 0) {
throw new AlgebricksException(
- "Something went wrong executing your Pregelix Job. Perhaps the Pregelix cluster needs to be restarted. "
+ "Something went wrong executing your Pregelix Job. Perhaps the Pregelix cluster "
+ + "needs to be restarted. "
+ "Check the following things: Are the datatypes of Asterix and Pregelix matching? "
- + "Is the server configuration correct (node names, buffer sizes, framesize)? Check the logfiles for more details.");
+ + "Is the server configuration correct (node names, buffer sizes, framesize)? "
+ + "Check the logfiles for more details.");
}
} catch (Exception e) {
if (bActiveTxn) {
@@ -2924,10 +2940,9 @@
}
// Prepares to run a program on external runtime.
- private void prepareRunExternalRuntime(AqlMetadataProvider metadataProvider, IHyracksClientConnection hcc,
+ protected void prepareRunExternalRuntime(AqlMetadataProvider metadataProvider, IHyracksClientConnection hcc,
RunStatement pregelixStmt, String dataverseNameFrom, String dataverseNameTo, String datasetNameFrom,
- String datasetNameTo, MetadataTransactionContext mdTxnCtx)
- throws AlgebricksException, AsterixException, Exception {
+ String datasetNameTo, MetadataTransactionContext mdTxnCtx) throws Exception {
// Validates the source/sink dataverses and datasets.
Dataset fromDataset = metadataProvider.findDataset(dataverseNameFrom, datasetNameFrom);
if (fromDataset == null) {
@@ -2955,8 +2970,8 @@
throw new AlgebricksException("Tried to access non-existing dataset: " + datasetNameTo);
}
// Cleans up the sink dataset -- Drop and then Create.
- DropStatement dropStmt =
- new DropStatement(new Identifier(dataverseNameTo), pregelixStmt.getDatasetNameTo(), true);
+ DropDatasetStatement dropStmt = new DropDatasetStatement(new Identifier(dataverseNameTo),
+ pregelixStmt.getDatasetNameTo(), true);
this.handleDatasetDropStatement(metadataProvider, dropStmt, hcc);
IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(),
toIndex.getKeyFieldSourceIndicators(), false, null, toDataset.getDatasetDetails().isTemp());
@@ -2969,7 +2984,7 @@
toDataset.getDatasetType(), idd, false);
this.handleCreateDatasetStatement(metadataProvider, createToDataset, hcc);
} catch (Exception e) {
- e.printStackTrace();
+ LOGGER.log(Level.WARNING, e.getMessage(), e);
throw new AlgebricksException("Error cleaning the result dataset. This should not happen.");
}
@@ -2979,7 +2994,7 @@
}
// Executes external shell commands.
- private int executeExternalShellProgram(ProcessBuilder pb)
+ protected int executeExternalShellProgram(ProcessBuilder pb)
throws IOException, AlgebricksException, InterruptedException {
Process process = pb.start();
try (BufferedReader in = new BufferedReader(new InputStreamReader(process.getInputStream()))) {
@@ -2990,27 +3005,27 @@
LOGGER.severe(line);
if (line.contains("Connection refused")) {
throw new AlgebricksException(
- "The connection to your Pregelix cluster was refused. Is it running? Is the port in the query correct?");
+ "The connection to your Pregelix cluster was refused. Is it running? "
+ + "Is the port in the query correct?");
}
if (line.contains("Could not find or load main class")) {
- throw new AlgebricksException(
- "The main class of your Pregelix query was not found. Is the path to your .jar file correct?");
+ throw new AlgebricksException("The main class of your Pregelix query was not found. "
+ + "Is the path to your .jar file correct?");
}
if (line.contains("ClassNotFoundException")) {
- throw new AlgebricksException(
- "The vertex class of your Pregelix query was not found. Does it exist? Is the spelling correct?");
+ throw new AlgebricksException("The vertex class of your Pregelix query was not found. "
+ + "Does it exist? Is the spelling correct?");
}
}
}
process.waitFor();
}
// Gets the exit value of the program.
- int resultState = process.exitValue();
- return resultState;
+ return process.exitValue();
}
// Constructs a Pregelix command line.
- private List<String> constructPregelixCommand(RunStatement pregelixStmt, String fromDataverseName,
+ protected List<String> constructPregelixCommand(RunStatement pregelixStmt, String fromDataverseName,
String fromDatasetName, String toDataverseName, String toDatasetName) {
// Constructs AsterixDB parameters, e.g., URL, source dataset and sink dataset.
AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
@@ -3028,7 +3043,7 @@
asterixdbParameterBuilder.append("pregelix.asterixdb.output.cleanup=false,");
// construct command
- List<String> cmds = new ArrayList<String>();
+ List<String> cmds = new ArrayList<>();
cmds.add("bin/pregelix");
cmds.add(pregelixStmt.getParameters().get(0)); // jar
cmds.add(pregelixStmt.getParameters().get(1)); // class
@@ -3073,7 +3088,7 @@
return cmds;
}
- private String getActiveDataverseName(String dataverse) throws AlgebricksException {
+ protected String getActiveDataverseName(String dataverse) throws AlgebricksException {
if (dataverse != null) {
return dataverse;
}
@@ -3083,14 +3098,14 @@
throw new AlgebricksException("dataverse not specified");
}
- private String getActiveDataverse(Identifier dataverse) throws AlgebricksException {
+ protected String getActiveDataverse(Identifier dataverse) throws AlgebricksException {
return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
}
- private void abort(Exception rootE, Exception parentE, MetadataTransactionContext mdTxnCtx) {
+ public static void abort(Exception rootE, Exception parentE, MetadataTransactionContext mdTxnCtx) {
try {
if (IS_DEBUG_MODE) {
- rootE.printStackTrace();
+ LOGGER.log(Level.SEVERE, rootE.getMessage(), rootE);
}
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
@@ -3099,7 +3114,7 @@
}
}
- private void rewriteStatement(Statement stmt) throws AsterixException {
+ protected void rewriteStatement(Statement stmt) throws AsterixException {
IStatementRewriter rewriter = rewriterFactory.createStatementRewriter();
rewriter.rewrite(stmt);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java
index 821822f..486b4ab 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java
@@ -25,6 +25,7 @@
import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.api.java.AsterixJavaClient;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -56,8 +57,8 @@
try {
for (String queryFile : options.args) {
Reader in = new FileReader(queryFile);
- AsterixJavaClient ajc = new AsterixJavaClient(
- integrationUtil.getHyracksClientConnection(), in, compilationProvider);
+ AsterixJavaClient ajc = new AsterixJavaClient(integrationUtil.getHyracksClientConnection(), in,
+ compilationProvider, new DefaultStatementExecutorFactory(null));
try {
ajc.compile(true, false, false, false, false, true, false);
} finally {
@@ -77,8 +78,9 @@
outdir.mkdirs();
File log = new File("asterix_logs");
- if (log.exists())
+ if (log.exists()) {
FileUtils.deleteDirectory(log);
+ }
File lsn = new File("last_checkpoint_lsn");
lsn.deleteOnExit();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java
index 2456bc2..e0a60c6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java
@@ -22,6 +22,7 @@
import org.apache.asterix.api.common.AsterixClientConfig;
import org.apache.asterix.api.java.AsterixJavaClient;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.hyracks.api.client.HyracksConnection;
@@ -62,7 +63,8 @@
boolean onlyPhysical, boolean createBinaryRuntime) throws Exception {
ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
FileReader reader = new FileReader(filename);
- AsterixJavaClient q = new AsterixJavaClient(hcc, reader, compilationProvider);
+ AsterixJavaClient q = new AsterixJavaClient(hcc, reader, compilationProvider,
+ new DefaultStatementExecutorFactory(null));
q.compile(optimize, true, true, true, onlyPhysical, createBinaryRuntime, createBinaryRuntime);
return q;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java
index f4c8062..8bcceca 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java
@@ -19,18 +19,22 @@
package org.apache.asterix.drivers;
import org.apache.asterix.api.http.servlet.APIServlet;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.compiler.provider.AqlCompilationProvider;
+import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHolder;
public class AsterixWebServer {
- public static void main(String args[]) throws Exception {
+ public static void main(String[] args) throws Exception {
Server server = new Server(8080);
ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
context.setContextPath("/");
server.setHandler(context);
- context.addServlet(new ServletHolder(new APIServlet()), "/*");
+ context.addServlet(new ServletHolder(new APIServlet(new AqlCompilationProvider(),
+ new SqlppCompilationProvider(), new DefaultStatementExecutorFactory(null))), "/*");
server.start();
server.join();
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 3ab6f3d..cf57174 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.hyracks.bootstrap;
+import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
import java.util.ArrayList;
import java.util.List;
import java.util.logging.Level;
@@ -39,6 +42,7 @@
import org.apache.asterix.api.http.servlet.ShutdownAPIServlet;
import org.apache.asterix.api.http.servlet.UpdateAPIServlet;
import org.apache.asterix.api.http.servlet.VersionAPIServlet;
+import org.apache.asterix.app.cc.CompilerExtensionManager;
import org.apache.asterix.app.external.ActiveLifecycleListener;
import org.apache.asterix.app.external.ExternalLibraryUtils;
import org.apache.asterix.common.api.AsterixThreadFactory;
@@ -47,8 +51,6 @@
import org.apache.asterix.common.config.AsterixMetadataProperties;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.utils.ServletUtil.Servlets;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
import org.apache.asterix.event.service.ILookupService;
import org.apache.asterix.external.library.ExternalLibraryManager;
import org.apache.asterix.messaging.CCMessageBroker;
@@ -69,9 +71,6 @@
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHolder;
-import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
private static final Logger LOGGER = Logger.getLogger(CCApplicationEntryPoint.class.getName());
@@ -80,6 +79,7 @@
private static IAsterixStateProxy proxy;
protected ICCApplicationContext appCtx;
+ protected CompilerExtensionManager ccExtensionManager;
@Override
public void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception {
@@ -91,22 +91,25 @@
}
appCtx.setThreadFactory(new AsterixThreadFactory(new LifeCycleComponentManager()));
- GlobalRecoveryManager.INSTANCE = new GlobalRecoveryManager((HyracksConnection) getNewHyracksClientConnection());
+ GlobalRecoveryManager.instantiate((HyracksConnection) getNewHyracksClientConnection());
ILibraryManager libraryManager = new ExternalLibraryManager();
ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
- AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection(), GlobalRecoveryManager.INSTANCE,
+ AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection(), GlobalRecoveryManager.instance(),
libraryManager);
+ ccExtensionManager = new CompilerExtensionManager(
+ AsterixAppContextInfo.getInstance().getExtensionProperties().getExtensions());
+ AsterixAppContextInfo.getInstance().setExtensionManager(ccExtensionManager);
if (System.getProperty("java.rmi.server.hostname") == null) {
System.setProperty("java.rmi.server.hostname",
((ClusterControllerService) ccAppCtx.getControllerService()).getCCConfig().clusterNetIpAddress);
}
- proxy = AsterixStateProxy.registerRemoteObject();
+ setAsterixStateProxy(AsterixStateProxy.registerRemoteObject());
appCtx.setDistributedState(proxy);
AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
- MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
+ MetadataManager.instantiate(new MetadataManager(proxy, metadataProperties));
AsterixAppContextInfo.getInstance().getCCApplicationContext()
.addJobLifecycleListener(ActiveLifecycleListener.INSTANCE);
@@ -117,7 +120,7 @@
server.start();
}
- ClusterManager.INSTANCE.registerSubscriber(GlobalRecoveryManager.INSTANCE);
+ ClusterManager.INSTANCE.registerSubscriber(GlobalRecoveryManager.instance());
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
ccAppCtx.setMessageBroker(messageBroker);
@@ -168,7 +171,9 @@
context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
webServer.setHandler(context);
- context.addServlet(new ServletHolder(new APIServlet()), "/*");
+ context.addServlet(new ServletHolder(new APIServlet(ccExtensionManager.getAqlCompilationProvider(),
+ ccExtensionManager.getSqlppCompilationProvider(), ccExtensionManager.getQueryTranslatorFactory())),
+ "/*");
return webServer;
}
@@ -235,27 +240,36 @@
protected Servlet createServlet(Servlets key) {
switch (key) {
case AQL:
- return new AQLAPIServlet(new AqlCompilationProvider());
+ return new AQLAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case AQL_QUERY:
- return new QueryAPIServlet(new AqlCompilationProvider());
+ return new QueryAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case AQL_UPDATE:
- return new UpdateAPIServlet(new AqlCompilationProvider());
+ return new UpdateAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case AQL_DDL:
- return new DDLAPIServlet(new AqlCompilationProvider());
+ return new DDLAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case SQLPP:
- return new AQLAPIServlet(new SqlppCompilationProvider());
+ return new AQLAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case SQLPP_QUERY:
- return new QueryAPIServlet(new SqlppCompilationProvider());
+ return new QueryAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case SQLPP_UPDATE:
- return new UpdateAPIServlet(new SqlppCompilationProvider());
+ return new UpdateAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case SQLPP_DDL:
- return new DDLAPIServlet(new SqlppCompilationProvider());
+ return new DDLAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case QUERY_STATUS:
return new QueryStatusAPIServlet();
case QUERY_RESULT:
return new QueryResultAPIServlet();
case QUERY_SERVICE:
- return new QueryServiceServlet(new SqlppCompilationProvider());
+ return new QueryServiceServlet(ccExtensionManager.getSqlppCompilationProvider(),
+ ccExtensionManager.getQueryTranslatorFactory());
case CONNECTOR:
return new ConnectorAPIServlet();
case SHUTDOWN:
@@ -294,4 +308,8 @@
ClusterState.ACTIVE);
}
}
-}
\ No newline at end of file
+
+ public static synchronized void setAsterixStateProxy(IAsterixStateProxy proxy) {
+ CCApplicationEntryPoint.proxy = proxy;
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index af8aa31..0d5cfb2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -32,7 +32,6 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -41,6 +40,7 @@
import org.apache.asterix.metadata.entities.Dataverse;
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.util.AsterixClusterProperties;
import org.apache.hyracks.api.client.HyracksConnection;
import org.apache.hyracks.api.job.JobId;
@@ -48,19 +48,19 @@
public class GlobalRecoveryManager implements IGlobalRecoveryMaanger {
- private static ClusterState state;
private static final Logger LOGGER = Logger.getLogger(GlobalRecoveryManager.class.getName());
+ private static GlobalRecoveryManager instance;
+ private static ClusterState state;
private HyracksConnection hcc;
- public static GlobalRecoveryManager INSTANCE;
- public GlobalRecoveryManager(HyracksConnection hcc) throws Exception {
- state = ClusterState.UNUSABLE;
+ private GlobalRecoveryManager(HyracksConnection hcc) {
+ setState(ClusterState.UNUSABLE);
this.hcc = hcc;
}
@Override
public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
- state = AsterixClusterProperties.INSTANCE.getState();
+ setState(AsterixClusterProperties.INSTANCE.getState());
AsterixClusterProperties.INSTANCE.setGlobalRecoveryCompleted(false);
return Collections.emptySet();
}
@@ -115,7 +115,7 @@
// Get indexes
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
dataset.getDataverseName(), dataset.getDatasetName());
- if (indexes.size() > 0) {
+ if (!indexes.isEmpty()) {
// Get the state of the dataset
ExternalDatasetDetails dsd = (ExternalDatasetDetails) dataset
.getDatasetDetails();
@@ -217,8 +217,20 @@
LOGGER.info("Global Recovery Completed");
}
});
- state = newState;
+ setState(newState);
recoveryThread.start();
}
}
+
+ public static GlobalRecoveryManager instance() {
+ return instance;
+ }
+
+ public static synchronized void instantiate(HyracksConnection hcc) {
+ instance = new GlobalRecoveryManager(hcc);
+ }
+
+ public static synchronized void setState(ClusterState state) {
+ GlobalRecoveryManager.state = state;
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 4cb9d70..5abe6bc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -25,8 +25,8 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.api.common.AsterixAppRuntimeContext;
import org.apache.asterix.app.external.ExternalLibraryUtils;
+import org.apache.asterix.app.nc.AsterixNCAppRuntimeContext;
import org.apache.asterix.common.api.AsterixThreadFactory;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.config.AsterixMetadataProperties;
@@ -100,8 +100,7 @@
((NodeControllerService) ncAppCtx.getControllerService())
.getConfiguration().clusterNetPublicIPAddress);
}
-
- runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext, metadataRmiPort);
+ runtimeContext = new AsterixNCAppRuntimeContext(ncApplicationContext, metadataRmiPort);
AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider) runtimeContext)
.getMetadataProperties();
if (!metadataProperties.getNodeNames().contains(ncApplicationContext.getNodeId())) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
index 138b620..27a5365 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
@@ -25,7 +25,7 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.active.ActivePartitionMessage;
+import org.apache.asterix.active.message.ActivePartitionMessage;
import org.apache.asterix.app.external.ActiveLifecycleListener;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
@@ -81,7 +81,7 @@
case COMPLETE_FAILBACK_RESPONSE:
handleCompleteFailbcakResponse(message);
break;
- case ACTIVE_ENTITY_MESSAGE:
+ case ACTIVE_ENTITY_TO_CC_MESSAGE:
handleActiveEntityMessage(message);
break;
default:
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
index 13b0189..74a5ba2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
@@ -24,6 +24,8 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.active.message.ActiveManagerMessage;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
@@ -113,6 +115,9 @@
case REPLICA_EVENT:
handleReplicaEvent(message);
break;
+ case ACTIVE_MANAGER_MESSAGE:
+ ((ActiveManager) appContext.getActiveManager()).submit((ActiveManagerMessage) message);
+ break;
default:
break;
}
@@ -143,8 +148,8 @@
appContext.initializeMetadata(false);
appContext.exportMetadataNodeStub();
} finally {
- TakeoverMetadataNodeResponseMessage reponse = new TakeoverMetadataNodeResponseMessage(
- appContext.getTransactionSubsystem().getId());
+ TakeoverMetadataNodeResponseMessage reponse =
+ new TakeoverMetadataNodeResponseMessage(appContext.getTransactionSubsystem().getId());
sendMessage(reponse, null);
}
}
@@ -189,15 +194,15 @@
}
//mark the partitions to be closed as inactive
- PersistentLocalResourceRepository localResourceRepo = (PersistentLocalResourceRepository) appContext
- .getLocalResourceRepository();
+ PersistentLocalResourceRepository localResourceRepo =
+ (PersistentLocalResourceRepository) appContext.getLocalResourceRepository();
for (Integer partitionId : msg.getPartitions()) {
localResourceRepo.addInactivePartition(partitionId);
}
//send response after partitions prepared for failback
- PreparePartitionsFailbackResponseMessage reponse = new PreparePartitionsFailbackResponseMessage(msg.getPlanId(),
- msg.getRequestId(), msg.getPartitions());
+ PreparePartitionsFailbackResponseMessage reponse =
+ new PreparePartitionsFailbackResponseMessage(msg.getPlanId(), msg.getRequestId(), msg.getPartitions());
sendMessage(reponse, null);
}
@@ -207,8 +212,8 @@
IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
remoteRecoeryManager.completeFailbackProcess();
} finally {
- CompleteFailbackResponseMessage reponse = new CompleteFailbackResponseMessage(msg.getPlanId(),
- msg.getRequestId(), msg.getPartitions());
+ CompleteFailbackResponseMessage reponse =
+ new CompleteFailbackResponseMessage(msg.getPlanId(), msg.getRequestId(), msg.getPartitions());
sendMessage(reponse, null);
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 596dc41..7e35f11 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -23,9 +23,8 @@
import java.util.logging.Logger;
import org.apache.asterix.algebra.operators.physical.CommitRuntime;
-import org.apache.asterix.api.common.AsterixAppRuntimeContext;
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.app.external.TestLibrarian;
+import org.apache.asterix.app.nc.AsterixNCAppRuntimeContext;
import org.apache.asterix.common.api.ILocalResourceMetadata;
import org.apache.asterix.common.config.AsterixTransactionProperties;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
@@ -97,8 +96,8 @@
protected static final Logger LOGGER = Logger.getLogger(TestNodeController.class.getName());
protected static final String PATH_ACTUAL = "unittest" + File.separator;
- protected static final String PATH_BASE =
- StringUtils.join(new String[] { "src", "test", "resources", "nodetests" }, File.separator);
+ protected static final String PATH_BASE = StringUtils.join(new String[] { "src", "test", "resources", "nodetests" },
+ File.separator);
protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
protected static AsterixTransactionProperties txnProperties;
@@ -158,18 +157,18 @@
PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
mergePolicyFactory, mergePolicyProperties, filterFields);
IndexOperation op = IndexOperation.INSERT;
- IModificationOperationCallbackFactory modOpCallbackFactory =
- new PrimaryIndexModificationOperationCallbackFactory(getTxnJobId(), dataset.getDatasetId(),
- primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, op, ResourceType.LSM_BTREE, true);
- AsterixLSMTreeInsertDeleteOperatorDescriptor indexOpDesc =
- getInsertOpratorDesc(primaryIndexInfo, modOpCallbackFactory);
- LSMBTreeDataflowHelperFactory dataflowHelperFactory =
- getPrimaryIndexDataflowHelperFactory(ctx, primaryIndexInfo);
+ IModificationOperationCallbackFactory modOpCallbackFactory = new PrimaryIndexModificationOperationCallbackFactory(
+ getTxnJobId(), dataset.getDatasetId(), primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, op,
+ ResourceType.LSM_BTREE, true);
+ AsterixLSMTreeInsertDeleteOperatorDescriptor indexOpDesc = getInsertOpratorDesc(primaryIndexInfo,
+ modOpCallbackFactory);
+ LSMBTreeDataflowHelperFactory dataflowHelperFactory = getPrimaryIndexDataflowHelperFactory(ctx,
+ primaryIndexInfo);
Mockito.when(indexOpDesc.getIndexDataflowHelperFactory()).thenReturn(dataflowHelperFactory);
IRecordDescriptorProvider recordDescProvider = primaryIndexInfo.getInsertRecordDescriptorProvider();
- AsterixLSMInsertDeleteOperatorNodePushable insertOp =
- new AsterixLSMInsertDeleteOperatorNodePushable(indexOpDesc, ctx, PARTITION,
- primaryIndexInfo.primaryIndexInsertFieldsPermutations, recordDescProvider, op, true);
+ AsterixLSMInsertDeleteOperatorNodePushable insertOp = new AsterixLSMInsertDeleteOperatorNodePushable(
+ indexOpDesc, ctx, PARTITION, primaryIndexInfo.primaryIndexInsertFieldsPermutations, recordDescProvider,
+ op, true);
CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(), dataset.getDatasetId(),
primaryIndexInfo.primaryKeyIndexes, false, true, PARTITION);
insertOp.setOutputFrameWriter(0, commitOp, primaryIndexInfo.rDesc);
@@ -185,8 +184,8 @@
JobSpecification spec = new JobSpecification();
PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
mergePolicyFactory, mergePolicyProperties, filterFields);
- LSMBTreeDataflowHelperFactory indexDataflowHelperFactory =
- getPrimaryIndexDataflowHelperFactory(ctx, primaryIndexInfo);
+ LSMBTreeDataflowHelperFactory indexDataflowHelperFactory = getPrimaryIndexDataflowHelperFactory(ctx,
+ primaryIndexInfo);
BTreeSearchOperatorDescriptor searchOpDesc = new BTreeSearchOperatorDescriptor(spec, primaryIndexInfo.rDesc,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
primaryIndexInfo.fileSplitProvider, primaryIndexInfo.primaryIndexTypeTraits,
@@ -214,8 +213,8 @@
public AsterixLSMTreeInsertDeleteOperatorDescriptor getInsertOpratorDesc(PrimaryIndexInfo primaryIndexInfo,
IModificationOperationCallbackFactory modOpCallbackFactory) {
- AsterixLSMTreeInsertDeleteOperatorDescriptor indexOpDesc =
- Mockito.mock(AsterixLSMTreeInsertDeleteOperatorDescriptor.class);
+ AsterixLSMTreeInsertDeleteOperatorDescriptor indexOpDesc = Mockito
+ .mock(AsterixLSMTreeInsertDeleteOperatorDescriptor.class);
Mockito.when(indexOpDesc.getLifecycleManagerProvider())
.thenReturn(AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
Mockito.when(indexOpDesc.getStorageManager()).thenReturn(AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
@@ -258,12 +257,12 @@
int[] primaryIndexBloomFilterKeyFields, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields) {
- ILocalResourceMetadata localResourceMetadata =
- new LSMBTreeLocalResourceMetadata(primaryIndexTypeTraits, primaryIndexComparatorFactories,
- primaryIndexBloomFilterKeyFields, true, dataset.getDatasetId(), mergePolicyFactory,
- mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields);
- ILocalResourceFactoryProvider localResourceFactoryProvider =
- new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.LSMBTreeResource);
+ ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(primaryIndexTypeTraits,
+ primaryIndexComparatorFactories, primaryIndexBloomFilterKeyFields, true, dataset.getDatasetId(),
+ mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
+ filterFields);
+ ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+ localResourceMetadata, LocalResource.LSMBTreeResource);
return localResourceFactoryProvider;
}
@@ -277,8 +276,8 @@
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
BLOOM_FILTER_FALSE_POSITIVE_RATE, true, primaryIndexInfo.filterTypeTraits,
primaryIndexInfo.filterCmpFactories, primaryIndexInfo.btreeFields, primaryIndexInfo.filterFields, true);
- IndexDataflowHelper dataflowHelper =
- dataflowHelperFactory.createIndexDataflowHelper(indexOpDesc, ctx, PARTITION);
+ IndexDataflowHelper dataflowHelper = dataflowHelperFactory.createIndexDataflowHelper(indexOpDesc, ctx,
+ PARTITION);
return (LSMBTreeDataflowHelper) dataflowHelper;
}
@@ -309,8 +308,8 @@
PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
mergePolicyFactory, mergePolicyProperties, filterFields);
TreeIndexCreateOperatorDescriptor indexOpDesc = getIndexCreateOpDesc(primaryIndexInfo);
- LSMBTreeDataflowHelper dataflowHelper =
- getPrimaryIndexDataflowHelper(createTestContext(), primaryIndexInfo, indexOpDesc);
+ LSMBTreeDataflowHelper dataflowHelper = getPrimaryIndexDataflowHelper(createTestContext(), primaryIndexInfo,
+ indexOpDesc);
dataflowHelper.create();
}
@@ -323,11 +322,10 @@
}
private IBinaryComparatorFactory[] createPrimaryIndexComparatorFactories(IAType[] primaryKeyTypes) {
- IBinaryComparatorFactory[] primaryIndexComparatorFactories =
- new IBinaryComparatorFactory[primaryKeyTypes.length];
+ IBinaryComparatorFactory[] primaryIndexComparatorFactories = new IBinaryComparatorFactory[primaryKeyTypes.length];
for (int j = 0; j < primaryKeyTypes.length; ++j) {
- primaryIndexComparatorFactories[j] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(primaryKeyTypes[j], true);
+ primaryIndexComparatorFactories[j] = AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(primaryKeyTypes[j], true);
}
return primaryIndexComparatorFactories;
}
@@ -337,8 +335,8 @@
int i = 0;
ISerializerDeserializer<?>[] primaryIndexSerdes = new ISerializerDeserializer<?>[primaryIndexNumOfTupleFields];
for (; i < primaryKeyTypes.length; i++) {
- primaryIndexSerdes[i] =
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
+ primaryIndexSerdes[i] = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(primaryKeyTypes[i]);
}
primaryIndexSerdes[i++] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(recordType);
if (metaType != null) {
@@ -372,7 +370,7 @@
}
public TransactionSubsystem getTransactionSubsystem() {
- return (TransactionSubsystem) ((AsterixAppRuntimeContext) ExecutionTestUtil.integrationUtil.ncs[0]
+ return (TransactionSubsystem) ((AsterixNCAppRuntimeContext) ExecutionTestUtil.integrationUtil.ncs[0]
.getApplicationContext().getApplicationObject()).getTransactionSubsystem();
}
@@ -380,8 +378,8 @@
return getTransactionSubsystem().getTransactionManager();
}
- public AsterixAppRuntimeContext getAppRuntimeContext() {
- return (AsterixAppRuntimeContext) ExecutionTestUtil.integrationUtil.ncs[0].getApplicationContext()
+ public AsterixNCAppRuntimeContext getAppRuntimeContext() {
+ return (AsterixNCAppRuntimeContext) ExecutionTestUtil.integrationUtil.ncs[0].getApplicationContext()
.getApplicationObject();
}
@@ -423,8 +421,8 @@
this.mergePolicyProperties = mergePolicyProperties;
this.filterFields = filterFields;
primaryIndexNumOfTupleFields = primaryKeyTypes.length + (1 + ((metaType == null) ? 0 : 1));
- primaryIndexTypeTraits =
- createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, primaryKeyTypes, recordType, metaType);
+ primaryIndexTypeTraits = createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, primaryKeyTypes,
+ recordType, metaType);
primaryIndexComparatorFactories = createPrimaryIndexComparatorFactories(primaryKeyTypes);
primaryIndexBloomFilterKeyFields = createPrimaryIndexBloomFilterFields(primaryKeyTypes.length);
filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recordType);
@@ -435,8 +433,8 @@
primaryIndexComparatorFactories, primaryIndexBloomFilterKeyFields, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields);
fileSplitProvider = getFileSplitProvider(dataset);
- primaryIndexSerdes =
- createPrimaryIndexSerdes(primaryIndexNumOfTupleFields, primaryKeyTypes, recordType, metaType);
+ primaryIndexSerdes = createPrimaryIndexSerdes(primaryIndexNumOfTupleFields, primaryKeyTypes, recordType,
+ metaType);
rDesc = new RecordDescriptor(primaryIndexSerdes, primaryIndexTypeTraits);
primaryIndexInsertFieldsPermutations = new int[primaryIndexNumOfTupleFields];
for (int i = 0; i < primaryIndexNumOfTupleFields; i++) {
@@ -459,8 +457,8 @@
ISerializerDeserializer<?>[] primaryKeySerdes = new ISerializerDeserializer<?>[primaryKeyTypes.length];
for (int i = 0; i < primaryKeyTypes.length; i++) {
primaryKeyTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i]);
- primaryKeySerdes[i] =
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
+ primaryKeySerdes[i] = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(primaryKeyTypes[i]);
}
RecordDescriptor searcgRecDesc = new RecordDescriptor(primaryKeySerdes, primaryKeyTypeTraits);
IRecordDescriptorProvider rDescProvider = Mockito.mock(IRecordDescriptorProvider.class);
@@ -472,10 +470,10 @@
public RecordDescriptor getSearchOutputDesc(IAType[] keyTypes, ARecordType recordType, ARecordType metaType) {
int primaryIndexNumOfTupleFields = keyTypes.length + (1 + ((metaType == null) ? 0 : 1));
- ITypeTraits[] primaryIndexTypeTraits =
- createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, keyTypes, recordType, metaType);
- ISerializerDeserializer<?>[] primaryIndexSerdes =
- createPrimaryIndexSerdes(primaryIndexNumOfTupleFields, keyTypes, recordType, metaType);
+ ITypeTraits[] primaryIndexTypeTraits = createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, keyTypes,
+ recordType, metaType);
+ ISerializerDeserializer<?>[] primaryIndexSerdes = createPrimaryIndexSerdes(primaryIndexNumOfTupleFields,
+ keyTypes, recordType, metaType);
return new RecordDescriptor(primaryIndexSerdes, primaryIndexTypeTraits);
}
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java
index 532cea9..3591509 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java
@@ -28,7 +28,8 @@
import java.util.Arrays;
import java.util.List;
-import org.apache.asterix.api.common.SessionConfig;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.common.app.SessionConfig;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.event.schema.cluster.Cluster;
@@ -37,6 +38,7 @@
import org.apache.asterix.lang.common.statement.RunStatement;
import org.apache.asterix.om.util.AsterixAppContextInfo;
import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.translator.IStatementExecutor;
import org.junit.Assert;
import org.junit.Test;
@@ -67,7 +69,7 @@
when(mockCluster.getMasterNode()).thenReturn(mockMasterNode);
when(mockMasterNode.getClientIp()).thenReturn("127.0.0.1");
- QueryTranslator aqlTranslator = new QueryTranslator(statements, mockSessionConfig,
+ IStatementExecutor aqlTranslator = new DefaultStatementExecutorFactory(null).create(statements, mockSessionConfig,
new AqlCompilationProvider());
List<String> parameters = new ArrayList<String>();
parameters.add("examples/pregelix-example-jar-with-dependencies.jar");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java
index 84fb2b3..34dab28 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java
@@ -27,6 +27,7 @@
import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.api.java.AsterixJavaClient;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.test.aql.TestExecutor;
@@ -59,9 +60,8 @@
integrationUtil.init(true);
Reader loadReader = new BufferedReader(
new InputStreamReader(new FileInputStream(LOAD_FOR_ENLIST_FILE), "UTF-8"));
- AsterixJavaClient asterixLoad = new AsterixJavaClient(
- integrationUtil.getHyracksClientConnection(), loadReader, ERR,
- new AqlCompilationProvider());
+ AsterixJavaClient asterixLoad = new AsterixJavaClient(integrationUtil.getHyracksClientConnection(), loadReader,
+ ERR, new AqlCompilationProvider(), new DefaultStatementExecutorFactory(null));
try {
asterixLoad.compile(true, false, false, false, false, true, false);
} catch (AsterixException e) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index b29c0b0..1f64868 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -30,6 +30,7 @@
import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.api.java.AsterixJavaClient;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
@@ -171,7 +172,8 @@
ILangCompilationProvider provider = queryFile.getName().endsWith("aql") ? aqlCompilationProvider
: sqlppCompilationProvider;
IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
- AsterixJavaClient asterix = new AsterixJavaClient(hcc, query, plan, provider);
+ AsterixJavaClient asterix = new AsterixJavaClient(hcc, query, plan, provider,
+ new DefaultStatementExecutorFactory(null));
try {
asterix.compile(true, false, false, true, true, false, false);
} catch (AsterixException e) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index 545faed..3195c39 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -49,11 +49,15 @@
public static AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
public static List<ILibraryManager> setUp(boolean cleanup) throws Exception {
+ return setUp(cleanup, TEST_CONFIG_FILE_NAME);
+ }
+
+ public static List<ILibraryManager> setUp(boolean cleanup, String configFile) throws Exception {
System.out.println("Starting setup");
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting setup");
}
- System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+ System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, configFile);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("initializing pseudo cluster");
@@ -80,8 +84,8 @@
libraryManagers.add(AsterixAppContextInfo.getInstance().getLibraryManager());
// Adds library managers for NCs, one-per-NC.
for (NodeControllerService nc : integrationUtil.ncs) {
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) nc.getApplicationContext()
- .getApplicationObject();
+ IAsterixAppRuntimeContext runtimeCtx =
+ (IAsterixAppRuntimeContext) nc.getApplicationContext().getApplicationObject();
libraryManagers.add(runtimeCtx.getLibraryManager());
}
return libraryManagers;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
index d3dc541..74c7091 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
@@ -25,16 +25,17 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
/**
* Manages a Mini (local VM) HDFS cluster with a configured number of datanodes.
- * @author ramangrover29
*/
public class HDFSCluster {
@@ -90,10 +91,12 @@
Path destDir = new Path(HDFS_PATH);
dfs.mkdirs(destDir);
File srcDir = new File(localDataRoot + DATA_PATH);
- File[] listOfFiles = srcDir.listFiles();
- for (File srcFile : listOfFiles) {
- Path path = new Path(srcFile.getAbsolutePath());
- dfs.copyFromLocalFile(path, destDir);
+ if (srcDir.exists()) {
+ File[] listOfFiles = srcDir.listFiles();
+ for (File srcFile : listOfFiles) {
+ Path path = new Path(srcFile.getAbsolutePath());
+ dfs.copyFromLocalFile(path, destDir);
+ }
}
}
@@ -110,6 +113,7 @@
cleanupLocal();
}
}
+
public static void main(String[] args) throws Exception {
HDFSCluster cluster = new HDFSCluster();
cluster.setup();
@@ -125,10 +129,10 @@
String hdfsUrl = "hdfs://127.0.0.1:31888";
String hdfsPath = "/asterix/extrasmalltweets.txt";
conf.set("fs.default.name", hdfsUrl);
- conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+ conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName());
conf.setClassLoader(GenericAdapter.class.getClassLoader());
conf.set("mapred.input.dir", hdfsPath);
- conf.set("mapred.input.format.class", "org.apache.hadoop.mapred.TextInputFormat");
+ conf.set("mapred.input.format.class", TextInputFormat.class.getName());
return conf;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ExtensionId.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ExtensionId.java
new file mode 100644
index 0000000..1cd0ce7
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ExtensionId.java
@@ -0,0 +1,64 @@
+/*
+ * 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.common.api;
+
+import java.io.Serializable;
+
+import org.apache.commons.lang3.ObjectUtils;
+
+public class ExtensionId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private final String name;
+ private final int version;
+
+ public ExtensionId(String name, int version) {
+ this.name = name;
+ this.version = version;
+ }
+
+ @Override
+ public int hashCode() {
+ return ObjectUtils.hashCodeMulti(getName().hashCode(), version);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ } else if (o instanceof ExtensionId) {
+ ExtensionId oExtensionId = (ExtensionId) o;
+ return version == oExtensionId.version && ObjectUtils.equals(name, oExtensionId.getName());
+ }
+ return false;
+ }
+
+ public int getVersion() {
+ return version;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String toString() {
+ return name + ":" + version;
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
index 100da63..046d5c1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -74,7 +74,7 @@
public double getBloomFilterFalsePositiveRate();
- public Object getFeedManager();
+ public Object getActiveManager();
public IRemoteRecoveryManager getRemoteRecoveryManager();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
new file mode 100644
index 0000000..9551935
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.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.asterix.common.api;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * an interface for an extension that provides a mechanism to override system behaviour
+ * or add new features
+ */
+public interface IExtension {
+
+ enum ExtensionKind {
+ /**
+ * Extends Query translator
+ */
+ STATEMENT_EXECUTOR,
+ /**
+ * Extends Metadata
+ */
+ METADATA,
+ /**
+ * Extends Language Syntax and Algebric Operations
+ */
+ LANG
+ }
+
+ /**
+ * @return Unique Id for the extension (used for lookup operations)
+ */
+ ExtensionId getId();
+
+ /**
+ * Configures the extension with configuration parameters.
+ * This method is called on system boot
+ *
+ * @param args
+ */
+ void configure(List<Pair<String, String>> args);
+
+ /**
+ * @return The extension point implemented by this extension
+ */
+ ExtensionKind getExtensionKind();
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/SessionConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/app/SessionConfig.java
similarity index 93%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/SessionConfig.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/app/SessionConfig.java
index ba205d1..f4e3be2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/SessionConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/app/SessionConfig.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.api.common;
+package org.apache.asterix.common.app;
import java.io.PrintWriter;
import java.util.HashMap;
@@ -105,6 +105,7 @@
*/
public static final String FORMAT_QUOTE_RECORD = "quote-record";
+ @FunctionalInterface
public interface ResultDecorator {
AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException;
}
@@ -179,7 +180,7 @@
this.optimize = optimize;
this.executeQuery = executeQuery;
this.generateJobSpec = generateJobSpec;
- this.flags = new HashMap<String, Boolean>();
+ this.flags = new HashMap<>();
}
/**
@@ -228,13 +229,13 @@
/**
* Specify all out-of-band settings at once. For convenience of older code.
*/
- public void setOOBData(boolean expr_tree, boolean rewritten_expr_tree, boolean logical_plan,
- boolean optimized_logical_plan, boolean hyracks_job) {
- this.set(OOB_EXPR_TREE, expr_tree);
- this.set(OOB_REWRITTEN_EXPR_TREE, rewritten_expr_tree);
- this.set(OOB_LOGICAL_PLAN, logical_plan);
- this.set(OOB_OPTIMIZED_LOGICAL_PLAN, optimized_logical_plan);
- this.set(OOB_HYRACKS_JOB, hyracks_job);
+ public void setOOBData(boolean exprTree, boolean rewrittenExprTree, boolean logicalPlan,
+ boolean optimizedLogicalPlan, boolean hyracksJob) {
+ this.set(OOB_EXPR_TREE, exprTree);
+ this.set(OOB_REWRITTEN_EXPR_TREE, rewrittenExprTree);
+ this.set(OOB_LOGICAL_PLAN, logicalPlan);
+ this.set(OOB_OPTIMIZED_LOGICAL_PLAN, optimizedLogicalPlan);
+ this.set(OOB_HYRACKS_JOB, hyracksJob);
}
/**
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExtension.java
similarity index 61%
rename from asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExtension.java
index 8b0914a..7417bc6 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExtension.java
@@ -16,24 +16,26 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.active;
+package org.apache.asterix.common.config;
-public class ActiveRuntime implements IActiveRuntime {
+import java.util.List;
- /** A unique identifier for the runtime **/
- protected final ActiveRuntimeId runtimeId;
+import org.apache.hyracks.algebricks.common.utils.Pair;
- public ActiveRuntime(ActiveRuntimeId runtimeId) {
- this.runtimeId = runtimeId;;
+public class AsterixExtension {
+ private final String className;
+ private final List<Pair<String, String>> args;
+
+ public AsterixExtension(String className, List<Pair<String, String>> args) {
+ this.className = className;
+ this.args = args;
}
- @Override
- public ActiveRuntimeId getRuntimeId() {
- return runtimeId;
+ public List<Pair<String, String>> getArgs() {
+ return args;
}
- @Override
- public String toString() {
- return runtimeId.toString();
+ public String getClassName() {
+ return className;
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExtensionProperties.java
similarity index 69%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExtensionProperties.java
index 1752054..e8f2bcc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExtensionProperties.java
@@ -16,12 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.common.config;
-import java.util.Map;
+import java.util.List;
-public interface ISubscriberRuntime {
+public class AsterixExtensionProperties extends AbstractAsterixProperties {
- public Map<String, String> getFeedPolicy();
+ public AsterixExtensionProperties(AsterixPropertiesAccessor accessor) {
+ super(accessor);
+ }
+ public List<AsterixExtension> getExtensions() {
+ return accessor.getExtensions();
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixProperties.java
new file mode 100644
index 0000000..1576774
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixProperties.java
@@ -0,0 +1,54 @@
+/*
+ * 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.common.config;
+
+import java.io.File;
+
+public class AsterixProperties {
+ //---------------------------- Directories ---------------------------//
+ private static final String VAR = File.separator + "var";
+ private static final String LIB = VAR + File.separator + "lib";
+ private static final String ASTERIXDB = LIB + File.separator + "asterixdb";
+ //----------------------------- Sections -----------------------------//
+ public static final String SECTION_ASTERIX = "asterix";
+ public static final String SECTION_PREFIX_EXTENSION = "extension/";
+ public static final String SECTION_CC = "cc";
+ public static final String SECTION_PREFIX_NC = "nc/";
+ //---------------------------- Properties ---=------------------------//
+ public static final String PROPERTY_CLUSTER_ADDRESS = "cluster.address";
+ public static final String PROPERTY_INSTANCE_NAME = "instance";
+ public static final String DEFAULT_INSTANCE_NAME = "DEFAULT_INSTANCE";
+ public static final String PROPERTY_METADATA_PORT = "metadata.port";
+ public static final String PROPERTY_COREDUMP_DIR = "coredumpdir";
+ public static final String DEFAULT_COREDUMP_DIR = String.join(File.separator, ASTERIXDB, "coredump");
+ public static final String PROPERTY_TXN_LOG_DIR = "txnlogdir";
+ public static final String DEFAULT_TXN_LOG_DIR = String.join(File.separator, ASTERIXDB, "txn-log");
+ public static final String PROPERTY_IO_DEV = "iodevices";
+ public static final String DEFAULT_IO_DEV = String.join(File.separator, ASTERIXDB, "iodevice");
+ public static final String PROPERTY_STORAGE_DIR = "storagedir";
+ public static final String DEFAULT_STORAGE_DIR = "storage";
+ public static final String PROPERTY_CLASS = "class";
+
+ private AsterixProperties() {
+ }
+
+ public static final String getSectionId(String prefix, String section) {
+ return section.substring(prefix.length());
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
index 7309f0c..ea1ee31 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixPropertiesAccessor.java
@@ -28,10 +28,9 @@
import java.util.List;
import java.util.Map;
import java.util.Properties;
+import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
-import java.util.logging.Level;
-import java.util.logging.Logger;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
@@ -40,14 +39,20 @@
import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.configuration.AsterixConfiguration;
import org.apache.asterix.common.configuration.Coredump;
+import org.apache.asterix.common.configuration.Extension;
import org.apache.asterix.common.configuration.Property;
import org.apache.asterix.common.configuration.Store;
import org.apache.asterix.common.configuration.TransactionLogDir;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.utils.ConfigUtil;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.application.IApplicationConfig;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
public class AsterixPropertiesAccessor {
- private static Logger LOGGER = Logger.getLogger(AsterixPropertiesAccessor.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(AsterixPropertiesAccessor.class.getName());
private final String instanceName;
private final String metadataNodeName;
@@ -62,36 +67,22 @@
private final Map<String, String> asterixBuildProperties = new HashMap<>();
private final Map<String, ClusterPartition[]> nodePartitionsMap;
private final SortedMap<Integer, ClusterPartition> clusterPartitions = new TreeMap<>();
+ // For extensions
+ private final List<AsterixExtension> extensions;
/**
* Constructor which reads asterix-configuration.xml, the old way.
+ *
* @throws AsterixException
+ * @throws IOException
*/
- public AsterixPropertiesAccessor() throws AsterixException {
+ public AsterixPropertiesAccessor() throws AsterixException, IOException {
String fileName = System.getProperty(GlobalConfig.CONFIG_FILE_PROPERTY);
if (fileName == null) {
fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
}
-
- InputStream is = this.getClass().getClassLoader().getResourceAsStream(fileName);
- if (is == null) {
- try {
- fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
- is = new FileInputStream(fileName);
- } catch (FileNotFoundException fnf) {
- throw new AsterixException("Could not find configuration file " + fileName);
- }
- }
-
- AsterixConfiguration asterixConfiguration = null;
+ AsterixConfiguration asterixConfiguration = configure(fileName);
cfg = null;
- try {
- JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
- Unmarshaller unmarshaller = ctx.createUnmarshaller();
- asterixConfiguration = (AsterixConfiguration) unmarshaller.unmarshal(is);
- } catch (JAXBException e) {
- throw new AsterixException("Failed to read configuration file " + fileName);
- }
instanceName = asterixConfiguration.getInstanceName();
metadataNodeName = asterixConfiguration.getMetadataNode();
List<Store> configuredStores = asterixConfiguration.getStore();
@@ -117,7 +108,16 @@
nodePartitionsMap.put(store.getNcId(), nodePartitions);
nodeNames.add(store.getNcId());
}
- asterixConfigurationParams = new HashMap<String, Property>();
+
+ // Get extensions
+ extensions = new ArrayList<>();
+ if (asterixConfiguration.getExtensions() != null) {
+ for (Extension ext : asterixConfiguration.getExtensions().getExtension()) {
+ extensions.add(ConfigUtil.toAsterixExtension(ext));
+ }
+ }
+
+ asterixConfigurationParams = new HashMap<>();
for (Property p : asterixConfiguration.getProperty()) {
asterixConfigurationParams.put(p.getName(), p);
}
@@ -130,56 +130,56 @@
loadAsterixBuildProperties();
}
+ private AsterixConfiguration configure(String fileName) throws IOException, AsterixException {
+ try (InputStream is = this.getClass().getClassLoader().getResourceAsStream(fileName)) {
+ if (is != null) {
+ return configure(is, fileName);
+ }
+ }
+ try (FileInputStream is = new FileInputStream(fileName)) {
+ return configure(is, fileName);
+ } catch (FileNotFoundException fnf1) {
+ LOGGER.warn("Failed to get configuration file " + fileName + " as FileInputStream. FileNotFoundException");
+ LOGGER.warn("Attempting to get default configuration file " + GlobalConfig.DEFAULT_CONFIG_FILE_NAME
+ + " as FileInputStream");
+ try (FileInputStream fis = new FileInputStream(GlobalConfig.DEFAULT_CONFIG_FILE_NAME)) {
+ return configure(fis, GlobalConfig.DEFAULT_CONFIG_FILE_NAME);
+ } catch (FileNotFoundException fnf2) {
+ fnf1.addSuppressed(fnf2);
+ throw new AsterixException("Could not find configuration file " + fileName, fnf1);
+ }
+ }
+ }
+
+ private AsterixConfiguration configure(InputStream is, String fileName) throws AsterixException {
+ try {
+ JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
+ Unmarshaller unmarshaller = ctx.createUnmarshaller();
+ return (AsterixConfiguration) unmarshaller.unmarshal(is);
+ } catch (JAXBException e) {
+ throw new AsterixException("Failed to read configuration file " + fileName, e);
+ }
+ }
+
/**
* Constructor which wraps an IApplicationConfig.
*/
public AsterixPropertiesAccessor(IApplicationConfig cfg) throws AsterixException {
this.cfg = cfg;
- instanceName = cfg.getString("asterix", "instance", "DEFAULT_INSTANCE");
+ instanceName = cfg.getString(AsterixProperties.SECTION_ASTERIX, AsterixProperties.PROPERTY_INSTANCE_NAME,
+ AsterixProperties.DEFAULT_INSTANCE_NAME);
String mdNode = null;
nodePartitionsMap = new HashMap<>();
- int uniquePartitionId = 0;
-
+ MutableInt uniquePartitionId = new MutableInt(0);
+ extensions = new ArrayList<>();
// Iterate through each configured NC.
for (String section : cfg.getSections()) {
- if (!section.startsWith("nc/")) {
- continue;
+ if (section.startsWith(AsterixProperties.SECTION_PREFIX_NC)) {
+ mdNode = configureNc(section, mdNode, uniquePartitionId);
+ } else if (section.startsWith(AsterixProperties.SECTION_PREFIX_EXTENSION)) {
+ String className = AsterixProperties.getSectionId(AsterixProperties.SECTION_PREFIX_EXTENSION, section);
+ configureExtension(className, section);
}
- String ncId = section.substring(3);
-
- // Here we figure out which is the metadata node. If any NCs
- // declare "metadata.port", use that one; otherwise just use the first.
- if (mdNode == null) {
- mdNode = ncId;
- }
- if (cfg.getString(section, "metadata.port") != null) {
- // QQQ But we don't actually *honor* metadata.port yet!
- mdNode = ncId;
- }
-
- // Now we assign the coredump and txnlog directories for this node.
- // QQQ Default values? Should they be specified here? Or should there
- // be a default.ini? Certainly wherever they are, they should be platform-dependent.
- coredumpConfig.put(ncId, cfg.getString(section, "coredumpdir", "/var/lib/asterixdb/coredump"));
- transactionLogDirs.put(ncId, cfg.getString(section, "txnlogdir", "/var/lib/asterixdb/txn-log"));
-
- // Now we create an array of ClusterPartitions for all the partitions
- // on this NC.
- String[] iodevices = cfg.getString(section, "iodevices", "/var/lib/asterixdb/iodevice").split(",");
- String storageSubdir = cfg.getString(section, "storagedir", "storage");
- String[] nodeStores = new String[iodevices.length];
- ClusterPartition[] nodePartitions = new ClusterPartition[iodevices.length];
- for (int i = 0; i < nodePartitions.length; i++) {
- // Construct final storage path from iodevice dir + storage subdir.
- nodeStores[i] = iodevices[i] + File.separator + storageSubdir;
- // Create ClusterPartition instances for this NC.
- ClusterPartition partition = new ClusterPartition(uniquePartitionId++, ncId, i);
- clusterPartitions.put(partition.getPartitionId(), partition);
- nodePartitions[i] = partition;
- }
- stores.put(ncId, nodeStores);
- nodePartitionsMap.put(ncId, nodePartitions);
- nodeNames.add(ncId);
}
metadataNodeName = mdNode;
@@ -187,6 +187,58 @@
loadAsterixBuildProperties();
}
+ private void configureExtension(String className, String section) {
+ Set<String> keys = cfg.getKeys(section);
+ List<Pair<String, String>> kvs = new ArrayList<>();
+ for (String key : keys) {
+ String value = cfg.getString(section, key);
+ kvs.add(new Pair<>(key, value));
+ }
+ extensions.add(new AsterixExtension(className, kvs));
+ }
+
+ private String configureNc(String section, String mdNode, MutableInt uniquePartitionId) {
+ String ncId = AsterixProperties.getSectionId(AsterixProperties.SECTION_PREFIX_NC, section);
+ String newMetadataNode = mdNode;
+
+ // Here we figure out which is the metadata node. If any NCs
+ // declare "metadata.port", use that one; otherwise just use the first.
+ if (mdNode == null || cfg.getString(section, AsterixProperties.PROPERTY_METADATA_PORT) != null) {
+ // QQQ But we don't actually *honor* metadata.port yet!
+ newMetadataNode = ncId;
+ }
+
+ // Now we assign the coredump and txnlog directories for this node.
+ // QQQ Default values? Should they be specified here? Or should there
+ // be a default.ini? Certainly wherever they are, they should be platform-dependent.
+ coredumpConfig.put(ncId, cfg.getString(section, AsterixProperties.PROPERTY_COREDUMP_DIR,
+ AsterixProperties.DEFAULT_COREDUMP_DIR));
+ transactionLogDirs.put(ncId,
+ cfg.getString(section, AsterixProperties.PROPERTY_TXN_LOG_DIR, AsterixProperties.DEFAULT_TXN_LOG_DIR));
+
+ // Now we create an array of ClusterPartitions for all the partitions
+ // on this NC.
+ String[] iodevices = cfg.getString(section, AsterixProperties.PROPERTY_IO_DEV,
+ AsterixProperties.DEFAULT_IO_DEV).split(",");
+ String storageSubdir = cfg.getString(section, AsterixProperties.PROPERTY_STORAGE_DIR,
+ AsterixProperties.DEFAULT_STORAGE_DIR);
+ String[] nodeStores = new String[iodevices.length];
+ ClusterPartition[] nodePartitions = new ClusterPartition[iodevices.length];
+ for (int i = 0; i < nodePartitions.length; i++) {
+ // Construct final storage path from iodevice dir + storage subdir.
+ nodeStores[i] = iodevices[i] + File.separator + storageSubdir;
+ // Create ClusterPartition instances for this NC.
+ ClusterPartition partition = new ClusterPartition(uniquePartitionId.getValue(), ncId, i);
+ uniquePartitionId.increment();
+ clusterPartitions.put(partition.getPartitionId(), partition);
+ nodePartitions[i] = partition;
+ }
+ stores.put(ncId, nodeStores);
+ nodePartitionsMap.put(ncId, nodePartitions);
+ nodeNames.add(ncId);
+ return newMetadataNode;
+ }
+
private void loadAsterixBuildProperties() throws AsterixException {
Properties gitProperties = new Properties();
try {
@@ -242,14 +294,14 @@
try {
return interpreter.interpret(value);
} catch (IllegalArgumentException e) {
- if (LOGGER.isLoggable(Level.SEVERE)) {
- StringBuilder msg =
- new StringBuilder("Invalid property value '" + value + "' for property '" + property + "'.\n");
+ if (LOGGER.isEnabledFor(Level.ERROR)) {
+ StringBuilder msg = new StringBuilder(
+ "Invalid property value '" + value + "' for property '" + property + "'.\n");
if (p != null) {
msg.append("See the description: \n" + p.getDescription() + "\n");
}
msg.append("Default = " + defaultValue);
- LOGGER.severe(msg.toString());
+ LOGGER.error(msg.toString());
}
throw e;
}
@@ -271,4 +323,8 @@
public SortedMap<Integer, ClusterPartition> getClusterPartitions() {
return clusterPartitions;
}
+
+ public List<AsterixExtension> getExtensions() {
+ return extensions;
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 66a01e0..a56d6f6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -21,6 +21,8 @@
public class ErrorCode {
public static final String ASTERIX = "ASX";
public static final int ERROR_CASTING_FIELD = 0;
+ public static final int ERROR_EXTENSION_CONFLICT = 1;
+ public static final int ERROR_PARSE_ERROR = 2;
private ErrorCode() {
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
index 7940680..9742a6c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
@@ -36,7 +36,8 @@
COMPLETE_FAILBACK_REQUEST,
COMPLETE_FAILBACK_RESPONSE,
REPLICA_EVENT,
- ACTIVE_ENTITY_MESSAGE
+ ACTIVE_ENTITY_TO_CC_MESSAGE,
+ ACTIVE_MANAGER_MESSAGE
}
public abstract ApplicationMessageType getMessageType();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ConfigUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ConfigUtil.java
new file mode 100644
index 0000000..5db1693
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ConfigUtil.java
@@ -0,0 +1,41 @@
+/*
+ * 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.common.utils;
+
+import java.util.ArrayList;
+
+import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.configuration.Extension;
+import org.apache.asterix.common.configuration.Property;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public class ConfigUtil {
+
+ private ConfigUtil() {
+ }
+
+ public static AsterixExtension toAsterixExtension(Extension ext) {
+ String className = ext.getExtensionClassName();
+ ArrayList<Pair<String, String>> args = new ArrayList<>();
+ for (Property property : ext.getProperty()) {
+ args.add(new Pair<>(property.getName(), property.getValue()));
+ }
+ return new AsterixExtension(className, args);
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/resources/schema/asterix-conf.xsd b/asterixdb/asterix-common/src/main/resources/schema/asterix-conf.xsd
index bb99319..d6a35cd 100644
--- a/asterixdb/asterix-common/src/main/resources/schema/asterix-conf.xsd
+++ b/asterixdb/asterix-common/src/main/resources/schema/asterix-conf.xsd
@@ -56,6 +56,9 @@
<xs:element
name="txnLogDirPath"
type="xs:string" />
+ <xs:element
+ name="extensionClassName"
+ type="xs:string" />
<!-- definition of complex elements -->
<xs:element name="store">
@@ -95,6 +98,29 @@
</xs:complexType>
</xs:element>
+ <xs:element name="extension">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="mg:extensionClassName"
+ maxOccurs="1"
+ minOccurs="1"/>
+ <xs:element ref="mg:property"
+ minOccurs="0"
+ maxOccurs="unbounded" />
+ </xs:sequence>
+ </xs:complexType>
+ </xs:element>
+
+ <xs:element name="extensions">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="mg:extension"
+ minOccurs="0"
+ maxOccurs="unbounded">
+ </xs:element>
+ </xs:sequence>
+ </xs:complexType>
+ </xs:element>
<xs:element name="asterixConfiguration">
<xs:complexType>
@@ -117,6 +143,9 @@
<xs:element
ref="mg:transactionLogDir"
maxOccurs="unbounded" />
+ <xs:element ref="mg:extensions"
+ minOccurs="0"
+ maxOccurs="1" />
<xs:element
ref="mg:property"
minOccurs="0"
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
index 6159908..d4e3641 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
@@ -76,7 +76,7 @@
this.spiller =
fpa.spillToDiskOnCongestion() ? new FrameSpiller(ctx,
connectionId.getFeedId() + "_" + connectionId.getDatasetName() + "_"
- + runtimeId.getFeedRuntimeType() + "_" + runtimeId.getPartition(),
+ + runtimeId.getRuntimeName() + "_" + runtimeId.getPartition(),
fpa.getMaxSpillOnDisk()) : null;
this.exceptionHandler = new FeedExceptionHandler(ctx, fta);
this.fpa = fpa;
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
index 40d2500..c40fed6 100644
--- 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
@@ -158,7 +158,7 @@
}
private void handleIntakePartitionStarts(ActiveEvent message, ActiveJob jobInfo) {
- if (feedPipeline.get(message.getFeedId()).first.decrementAndGet() == 0) {
+ if (feedPipeline.get(message.getEntityId()).first.decrementAndGet() == 0) {
((FeedIntakeInfo) jobInfo).getIntakeFeedJoint().setState(State.ACTIVE);
jobInfo.setState(ActivityState.ACTIVE);
notifyFeedEventSubscribers(FeedLifecycleEvent.FEED_INTAKE_STARTED);
@@ -339,7 +339,7 @@
return locations;
}
- private void notifyFeedEventSubscribers(FeedLifecycleEvent event) {
+ private synchronized void notifyFeedEventSubscribers(FeedLifecycleEvent event) {
if (subscribers != null && !subscribers.isEmpty()) {
for (IFeedLifecycleEventSubscriber subscriber : subscribers) {
subscriber.handleFeedEvent(event);
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 9c7a319..6c04b2d 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
@@ -19,8 +19,8 @@
package org.apache.asterix.external.feed.runtime;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.util.ExternalDataExceptionUtils;
+import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.log4j.Logger;
/**
@@ -30,13 +30,12 @@
private static final Logger LOGGER = Logger.getLogger(AdapterExecutor.class.getName());
- private final DistributeFeedFrameWriter writer; // A writer that sends frames to multiple receivers (that can
+ private final IFrameWriter writer; // A writer that sends frames to multiple receivers (that can
// increase or decrease at any time)
- private final FeedAdapter adapter; // The adapter
+ private final FeedAdapter adapter; // The adapter
private final AdapterRuntimeManager adapterManager;// The runtime manager <-- two way visibility -->
- public AdapterExecutor(int partition, DistributeFeedFrameWriter writer, FeedAdapter adapter,
- AdapterRuntimeManager adapterManager) {
+ public AdapterExecutor(IFrameWriter writer, FeedAdapter adapter, AdapterRuntimeManager adapterManager) {
this.writer = writer;
this.adapter = adapter;
this.adapterManager = adapterManager;
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 f2f0747..424f2dc 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
@@ -24,7 +24,7 @@
import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
+import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.log4j.Logger;
/**
@@ -34,27 +34,26 @@
private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
- private final EntityId feedId; // (dataverse-feed)
+ private final EntityId feedId; // (dataverse-feed)
- private final FeedAdapter feedAdapter; // The adapter
+ private final FeedAdapter feedAdapter; // The adapter
- private final AdapterExecutor adapterExecutor; // The executor for the adapter
+ private final AdapterExecutor adapterExecutor; // The executor for the adapter
- private final int partition; // The partition number
+ private final int partition; // The partition number
- private final ExecutorService executorService; // Executor service to run/shutdown the adapter executor
+ private final ExecutorService executorService; // Executor service to run/shutdown the adapter executor
- private IngestionRuntime ingestionRuntime; // Runtime representing the ingestion stage of a feed
+ private IngestionRuntime ingestionRuntime; // Runtime representing the ingestion stage of a feed
private volatile boolean done = false;
private volatile boolean failed = false;
- public AdapterRuntimeManager(EntityId feedId, FeedAdapter feedAdapter, DistributeFeedFrameWriter writer,
- int partition) {
- this.feedId = feedId;
+ public AdapterRuntimeManager(EntityId entityId, FeedAdapter feedAdapter, IFrameWriter writer, int partition) {
+ this.feedId = entityId;
this.feedAdapter = feedAdapter;
this.partition = partition;
- this.adapterExecutor = new AdapterExecutor(partition, writer, feedAdapter, this);
+ this.adapterExecutor = new AdapterExecutor(writer, feedAdapter, this);
this.executorService = Executors.newSingleThreadExecutor();
}
@@ -82,7 +81,6 @@
// stop() returned false, we try to force shutdown
executorService.shutdownNow();
}
-
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
index 294642e..821a0b1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
@@ -20,13 +20,13 @@
import java.util.Map;
-import org.apache.asterix.active.ActiveRuntime;
import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.IActiveRuntime;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* Represents the feed runtime that collects feed tuples from another feed.
@@ -34,18 +34,19 @@
* intake job. For a secondary feed, tuples are collected from the intake/compute
* runtime associated with the source feed.
*/
-public class CollectionRuntime extends ActiveRuntime implements ISubscriberRuntime {
+public class CollectionRuntime implements IActiveRuntime {
- private final FeedConnectionId connectionId; // [Dataverse - Feed - Dataset]
- private final ISubscribableRuntime sourceRuntime; // Runtime that provides the data
- private final Map<String, String> feedPolicy; // Policy associated with the feed
- private final FeedFrameCollector frameCollector; // Collector that can be plugged into a frame distributor
+ private final FeedConnectionId connectionId; // [Dataverse - Feed - Dataset]
+ private final ISubscribableRuntime sourceRuntime; // Runtime that provides the data
+ private final Map<String, String> feedPolicy; // Policy associated with the feed
+ private final FeedFrameCollector frameCollector; // Collector that can be plugged into a frame distributor
private final IHyracksTaskContext ctx;
+ private final ActiveRuntimeId runtimeId;
public CollectionRuntime(FeedConnectionId connectionId, ActiveRuntimeId runtimeId,
ISubscribableRuntime sourceRuntime, Map<String, String> feedPolicy, IHyracksTaskContext ctx,
FeedFrameCollector frameCollector) {
- super(runtimeId);
+ this.runtimeId = runtimeId;
this.connectionId = connectionId;
this.sourceRuntime = sourceRuntime;
this.feedPolicy = feedPolicy;
@@ -68,7 +69,6 @@
|| frameCollector.getState().equals(FeedFrameCollector.State.HANDOVER);
}
- @Override
public Map<String, String> getFeedPolicy() {
return feedPolicy;
}
@@ -88,4 +88,13 @@
public IHyracksTaskContext getCtx() {
return ctx;
}
+
+ @Override
+ public ActiveRuntimeId getRuntimeId() {
+ return runtimeId;
+ }
+
+ @Override
+ public void stop() throws HyracksDataException, InterruptedException {
+ }
}
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
index 9661890..34237c4 100644
--- 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
@@ -22,7 +22,7 @@
import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.active.EntityId;
-import org.apache.asterix.external.feed.api.ISubscriberRuntime;
+import org.apache.asterix.active.IActiveRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
import org.apache.hyracks.api.comm.VSizeFrame;
@@ -37,9 +37,9 @@
private final IHyracksTaskContext ctx;
private int numSubscribers = 0;
- public IngestionRuntime(EntityId feedId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
+ public IngestionRuntime(EntityId entityId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
AdapterRuntimeManager adaptorRuntimeManager, IHyracksTaskContext ctx) {
- super(feedId, runtimeId, feedWriter);
+ super(entityId, runtimeId, feedWriter);
this.adapterRuntimeManager = adaptorRuntimeManager;
this.ctx = ctx;
}
@@ -53,7 +53,7 @@
TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE,
TaskUtils.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx), collectionRuntime.getCtx());
- adapterRuntimeManager.start();
+ start();
}
numSubscribers++;
if (LOGGER.isLoggable(Level.INFO)) {
@@ -65,7 +65,7 @@
public synchronized void unsubscribe(CollectionRuntime collectionRuntime) throws InterruptedException {
numSubscribers--;
if (numSubscribers == 0) {
- adapterRuntimeManager.stop();
+ stop();
}
subscribers.remove(collectionRuntime);
}
@@ -75,7 +75,7 @@
}
public void terminate() {
- for (ISubscriberRuntime subscriber : subscribers) {
+ for (IActiveRuntime subscriber : subscribers) {
try {
unsubscribe((CollectionRuntime) subscriber);
} catch (Exception e) {
@@ -86,4 +86,12 @@
}
}
+ public void start() {
+ adapterRuntimeManager.start();
+ }
+
+ @Override
+ public void stop() throws InterruptedException {
+ adapterRuntimeManager.stop();
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
index 423e599..fb70fdb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
@@ -22,25 +22,30 @@
import java.util.List;
import java.util.logging.Logger;
-import org.apache.asterix.active.ActiveRuntime;
import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveRuntime;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-public abstract class SubscribableRuntime extends ActiveRuntime implements ISubscribableRuntime {
+public abstract class SubscribableRuntime implements ISubscribableRuntime {
protected static final Logger LOGGER = Logger.getLogger(SubscribableRuntime.class.getName());
protected final EntityId feedId;
- protected final List<ISubscriberRuntime> subscribers;
+ protected final List<IActiveRuntime> subscribers;
protected final DistributeFeedFrameWriter dWriter;
+ protected final ActiveRuntimeId runtimeId;
public SubscribableRuntime(EntityId feedId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter dWriter) {
- super(runtimeId);
+ this.runtimeId = runtimeId;
this.feedId = feedId;
this.dWriter = dWriter;
- this.subscribers = new ArrayList<ISubscriberRuntime>();
+ this.subscribers = new ArrayList<>();
+ }
+
+ @Override
+ public ActiveRuntimeId getRuntimeId() {
+ return runtimeId;
}
public EntityId getFeedId() {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
index 533d119..7acb1f8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
@@ -23,6 +23,7 @@
import java.util.Date;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import org.apache.commons.lang3.ObjectUtils;
public class ExternalFile implements Serializable, Comparable<ExternalFile> {
@@ -118,21 +119,21 @@
}
@Override
+ public int hashCode() {
+ return ObjectUtils.hashCodeMulti(dataverseName, datasetName, fileName);
+ }
+
+ @Override
public boolean equals(Object obj) {
- if (obj == null)
- return false;
- if (obj == this)
+ if (obj == this) {
return true;
- if (!(obj instanceof ExternalFile))
+ }
+ if (!(obj instanceof ExternalFile)) {
return false;
+ }
ExternalFile anotherFile = (ExternalFile) obj;
- if (fileNumber != anotherFile.fileNumber)
- return false;
- if (!dataverseName.equals(anotherFile.dataverseName))
- return false;
- if (!datasetName.equals(anotherFile.datasetName))
- return false;
- return true;
+ return fileNumber == anotherFile.fileNumber && ObjectUtils.equals(dataverseName, anotherFile.getDataverseName())
+ && ObjectUtils.equals(datasetName, anotherFile.getDatasetName());
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
index 84c2cb4..266669b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
@@ -78,9 +78,9 @@
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
ActiveManager feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject()).getFeedManager();
+ .getApplicationContext().getApplicationObject()).getActiveManager();
ActiveRuntimeId sourceRuntimeId = new ActiveRuntimeId(sourceFeedId, subscriptionLocation.toString(), partition);
- ISubscribableRuntime sourceRuntime = (ISubscribableRuntime) feedManager.getSubscribableRuntime(sourceRuntimeId);
+ ISubscribableRuntime sourceRuntime = (ISubscribableRuntime) feedManager.getRuntime(sourceRuntimeId);
return new FeedCollectOperatorNodePushable(ctx, connectionId, feedPolicyProperties, partition, sourceRuntime);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
index 231fe99..ee9a186 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
@@ -21,8 +21,8 @@
import java.util.Map;
import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.active.ActivePartitionMessage;
import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.message.ActivePartitionMessage;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
@@ -46,7 +46,7 @@
private final FeedConnectionId connectionId;
private final Map<String, String> feedPolicy;
private final FeedPolicyAccessor policyAccessor;
- private final ActiveManager feedManager;
+ private final ActiveManager activeManager;
private final ISubscribableRuntime sourceRuntime;
private final IHyracksTaskContext ctx;
private CollectionRuntime collectRuntime;
@@ -59,8 +59,8 @@
this.sourceRuntime = sourceRuntime;
this.feedPolicy = feedPolicy;
this.policyAccessor = new FeedPolicyAccessor(feedPolicy);
- this.feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
- .getApplicationObject()).getFeedManager();
+ this.activeManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ .getApplicationObject()).getActiveManager();
}
@Override
@@ -72,20 +72,19 @@
FrameTupleAccessor tAccessor = new FrameTupleAccessor(recordDesc);
if (policyAccessor.bufferingEnabled()) {
writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, writer, policyAccessor, tAccessor,
- feedManager.getFramePool());
+ activeManager.getFramePool());
} else {
writer = new SyncFeedRuntimeInputHandler(ctx, writer, tAccessor);
}
collectRuntime = new CollectionRuntime(connectionId, runtimeId, sourceRuntime, feedPolicy, ctx,
new FeedFrameCollector(policyAccessor, writer, connectionId));
- feedManager.getActiveRuntimeRegistry().registerRuntime(collectRuntime);
+ activeManager.registerRuntime(collectRuntime);
sourceRuntime.subscribe(collectRuntime);
// Notify CC that Collection started
- ctx.sendApplicationMessageToCC(
- new ActivePartitionMessage(connectionId.getFeedId(), ctx.getJobletContext().getJobId(), null),
- null);
+ ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
+ ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED), null);
collectRuntime.waitTillCollectionOver();
- feedManager.getActiveRuntimeRegistry().deregisterRuntime(collectRuntime.getRuntimeId());
+ activeManager.deregisterRuntime(collectRuntime.getRuntimeId());
} catch (Exception e) {
throw new HyracksDataException(e);
}
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 ffa451b..7c8fe14 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
@@ -19,9 +19,9 @@
package org.apache.asterix.external.operators;
import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.active.ActivePartitionMessage;
import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.message.ActivePartitionMessage;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
@@ -62,7 +62,7 @@
@Override
public void initialize() throws HyracksDataException {
ActiveManager feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject()).getFeedManager();
+ .getApplicationContext().getApplicationObject()).getActiveManager();
AdapterRuntimeManager adapterRuntimeManager = null;
DistributeFeedFrameWriter frameDistributor = null;
IngestionRuntime ingestionRuntime = null;
@@ -80,8 +80,8 @@
ingestionRuntime = new IngestionRuntime(feedId, runtimeId, frameDistributor, adapterRuntimeManager, ctx);
feedManager.registerRuntime(ingestionRuntime);
// Notify FeedJobNotificationHandler that this provider is ready to receive subscription requests.
- ctx.sendApplicationMessageToCC(new ActivePartitionMessage(feedId, ctx.getJobletContext().getJobId(), null),
- null);
+ ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
+ ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED), null);
// open the distributor
open = true;
frameDistributor.open();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
index 5f92327..1dce6be 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
@@ -63,7 +63,7 @@
this.partition = partition;
IAsterixAppRuntimeContext runtimeCtx =
(IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
- this.feedManager = (ActiveManager) runtimeCtx.getFeedManager();
+ this.feedManager = (ActiveManager) runtimeCtx.getActiveManager();
}
@Override
@@ -98,8 +98,7 @@
EntityId sourceFeedId = endFeedMessage.getSourceFeedId();
ActiveRuntimeId subscribableRuntimeId =
new ActiveRuntimeId(sourceFeedId, FeedRuntimeType.INTAKE.toString(), partition);
- ISubscribableRuntime feedRuntime =
- (ISubscribableRuntime) feedManager.getSubscribableRuntime(subscribableRuntimeId);
+ ISubscribableRuntime feedRuntime = (ISubscribableRuntime) feedManager.getRuntime(subscribableRuntimeId);
AdapterRuntimeManager adapterRuntimeManager = ((IngestionRuntime) feedRuntime).getAdapterRuntimeManager();
adapterRuntimeManager.stop();
if (LOGGER.isLoggable(Level.INFO)) {
@@ -129,8 +128,7 @@
}
runtimeId = new ActiveRuntimeId(endFeedMessage.getSourceFeedId(), runtimeType.toString(), partition);
- CollectionRuntime feedRuntime =
- (CollectionRuntime) feedManager.getActiveRuntimeRegistry().getRuntime(runtimeId);
+ CollectionRuntime feedRuntime = (CollectionRuntime) feedManager.getRuntime(runtimeId);
if (feedRuntime != null) {
feedRuntime.getSourceRuntime().unsubscribe(feedRuntime);
}
@@ -150,11 +148,10 @@
ActiveRuntimeId feedSubscribableRuntimeId = new ActiveRuntimeId(connectionId.getFeedId(),
FeedRuntimeType.COMPUTE.toString(), partition);
ISubscribableRuntime feedRuntime =
- (ISubscribableRuntime) feedManager.getSubscribableRuntime(feedSubscribableRuntimeId);
+ (ISubscribableRuntime) feedManager.getRuntime(feedSubscribableRuntimeId);
runtimeId = new ActiveRuntimeId(endFeedMessage.getSourceFeedId(),
FeedRuntimeType.COMPUTE_COLLECT.toString(), partition);
- CollectionRuntime feedCollectionRuntime =
- (CollectionRuntime) feedManager.getActiveRuntimeRegistry().getRuntime(runtimeId);
+ CollectionRuntime feedCollectionRuntime = (CollectionRuntime) feedManager.getRuntime(runtimeId);
feedRuntime.unsubscribe(feedCollectionRuntime);
break;
default:
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
index 37a42a7..ff996aa 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
@@ -24,7 +24,6 @@
import java.util.logging.Logger;
import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.active.ActiveRuntime;
import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
@@ -62,12 +61,6 @@
private FeedPolicyEnforcer policyEnforcer;
/**
- * The Feed Runtime instance associated with the operator. Feed Runtime
- * captures the state of the operator while the feed is active.
- */
- private ActiveRuntime feedRuntime;
-
- /**
* A unique identifier for the feed instance. A feed instance represents
* the flow of data from a feed to a dataset.
**/
@@ -112,7 +105,7 @@
this.partition = partition;
this.connectionId = feedConnectionId;
this.feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
- .getApplicationObject()).getFeedManager();
+ .getApplicationObject()).getActiveManager();
this.message = new VSizeFrame(ctx);
TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
this.opDesc = feedMetaOperatorDescriptor;
@@ -142,8 +135,6 @@
} else {
writer = new SyncFeedRuntimeInputHandler(ctx, coreOperator, fta);
}
- feedRuntime = new ActiveRuntime(runtimeId);
- feedManager.getActiveRuntimeRegistry().registerRuntime(feedRuntime);
}
@Override
@@ -164,19 +155,11 @@
@Override
public void close() throws HyracksDataException {
- try {
- deregister();
- } finally {
- if (opened) {
- writer.close();
- }
+ if (opened) {
+ writer.close();
}
}
- private void deregister() {
- feedManager.getActiveRuntimeRegistry().deregisterRuntime(feedRuntime.getRuntimeId());
- }
-
@Override
public void flush() throws HyracksDataException {
writer.flush();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
index 95bebad..2c3b62f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
@@ -24,7 +24,6 @@
import java.util.logging.Logger;
import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.active.ActiveRuntime;
import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.dataflow.AsterixLSMInsertDeleteOperatorNodePushable;
@@ -34,7 +33,6 @@
import org.apache.asterix.external.feed.policy.FeedPolicyEnforcer;
import org.apache.asterix.external.util.FeedUtils;
import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IActivity;
@@ -60,12 +58,6 @@
private final FeedPolicyEnforcer policyEnforcer;
/**
- * The Feed Runtime instance associated with the operator. Feed Runtime
- * captures the state of the operator while the feed is active.
- */
- private ActiveRuntime feedRuntime;
-
- /**
* A unique identifier for the feed instance. A feed instance represents
* the flow of data from a feed to a dataset.
**/
@@ -106,7 +98,7 @@
this.partition = partition;
this.connectionId = feedConnectionId;
this.feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
- .getApplicationObject()).getFeedManager();
+ .getApplicationObject()).getActiveManager();
this.targetId = targetId;
this.message = new VSizeFrame(ctx);
TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
@@ -116,8 +108,8 @@
@Override
public void open() throws HyracksDataException {
- ActiveRuntimeId runtimeId =
- new ActiveRuntimeId(connectionId.getFeedId(), runtimeType.toString() + "." + targetId, partition);
+ ActiveRuntimeId runtimeId = new ActiveRuntimeId(connectionId.getFeedId(),
+ runtimeType.toString() + "." + targetId, partition);
try {
initializeNewFeedRuntime(runtimeId);
insertOperator.open();
@@ -135,7 +127,6 @@
(AsterixLSMInsertDeleteOperatorNodePushable) insertOperator;
if (!indexOp.isPrimary()) {
writer = insertOperator;
- setupBasicRuntime(writer);
return;
}
}
@@ -145,14 +136,6 @@
} else {
writer = new SyncFeedRuntimeInputHandler(ctx, insertOperator, fta);
}
- setupBasicRuntime(writer);
- }
-
- private void setupBasicRuntime(IFrameWriter frameWriter) throws Exception {
- ActiveRuntimeId runtimeId =
- new ActiveRuntimeId(connectionId.getFeedId(), runtimeType.toString() + "." + targetId, partition);
- feedRuntime = new ActiveRuntime(runtimeId);
- feedManager.getActiveRuntimeRegistry().registerRuntime(feedRuntime);
}
@Override
@@ -173,15 +156,7 @@
@Override
public void close() throws HyracksDataException {
- try {
- writer.close();
- } finally {
- deregister();
- }
- }
-
- private void deregister() {
- feedManager.getActiveRuntimeRegistry().deregisterRuntime(feedRuntime.getRuntimeId());
+ writer.close();
}
@Override
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
index 65f61f2..5edc521 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
@@ -112,8 +112,8 @@
buildOtherUdfs(topExpr.getBody(), otherFDecls, funIds);
declaredFunctions.addAll(otherFDecls);
if (!declaredFunctions.isEmpty()) {
- AQLInlineUdfsVisitor visitor = new AQLInlineUdfsVisitor(context, new AQLRewriterFactory(),
- declaredFunctions, metadataProvider);
+ AQLInlineUdfsVisitor visitor =
+ new AQLInlineUdfsVisitor(context, new AQLRewriterFactory(), declaredFunctions, metadataProvider);
while (topExpr.accept(visitor, declaredFunctions)) {
// loop until no more changes
}
@@ -154,9 +154,9 @@
FunctionDecl functionDecl = functionParser.getFunctionDecl(function);
if (functionDecl != null) {
if (functionDecls.contains(functionDecl)) {
- throw new AsterixException("Recursive invocation "
- + functionDecls.get(functionDecls.size() - 1).getSignature() + " <==> "
- + functionDecl.getSignature());
+ throw new AsterixException(
+ "Recursive invocation " + functionDecls.get(functionDecls.size() - 1).getSignature()
+ + " <==> " + functionDecl.getSignature());
}
functionDecls.add(functionDecl);
buildOtherUdfs(functionDecl.getFuncBody(), functionDecls, declaredFunctions);
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
index 836de6a..d4bf6bc 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
@@ -204,4 +204,9 @@
public String[] getLocations() {
return locations;
}
+
+ @Override
+ public byte getCategory() {
+ return Category.PROCEDURE;
+ }
}
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 9cffda0..f46f13d 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -43,7 +43,6 @@
import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.aql.clause.DistinctClause;
@@ -106,7 +105,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
@@ -130,6 +129,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
@@ -872,7 +872,7 @@
(
<DATASET> pairId = QualifiedName() ifExists = IfExists()
{
- stmt = new DropStatement(pairId.first, pairId.second, ifExists);
+ stmt = new DropDatasetStatement(pairId.first, pairId.second, ifExists);
}
| <INDEX> tripleId = DoubleQualifiedName() ifExists = IfExists()
{
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
index b1df329..7da34fe 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
@@ -38,7 +38,6 @@
UNION_EXPRESSION,
SELECT_EXPRESSION,
PRIMARY_EXPRESSION,
- VALUE_EXPRESSION,
INDEPENDENT_SUBQUERY,
CASE_EXPRESSION
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index fcc1080..b12195f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -34,6 +34,28 @@
*/
public byte getKind();
+ /**
+ * get a byte representing the statement category.
+ * Each category describes the type of modifications this statement does.
+ *
+ * @return kind byte
+ */
+ public byte getCategory();
+
+ public class Category {
+ /** no modifications */
+ public static final byte QUERY = 0x01;
+ /** modify data */
+ public static final byte UPDATE = 0x02;
+ /** modify metadata */
+ public static final byte DDL = 0x04;
+ /** modify anything */
+ public static final byte PROCEDURE = 0x08;
+
+ private Category() {
+ }
+ }
+
public class Kind {
public static final byte DATASET_DECL = 0x00;
public static final byte DATAVERSE_DECL = 0x01;
@@ -70,6 +92,7 @@
public static final byte COMPACT = 0x20;
public static final byte EXTERNAL_DATASET_REFRESH = 0x21;
public static final byte RUN = 0x22;
+ public static final byte EXTENSION = 0x23;
private Kind() {
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
index 912ac6f..94aa1ac 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
@@ -50,4 +50,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
index 48bc59e..fd1ca5e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
@@ -102,4 +102,9 @@
return feedName;
}
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
index c6dff3f..a475051 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
@@ -57,4 +57,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
index dfc6c7e..75fc556 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
@@ -87,4 +87,9 @@
return description;
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
index c4f9efb..54b6a94 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
@@ -57,4 +57,8 @@
@Override
public abstract <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException;
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
index 02a2b0c..5a75f43 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
@@ -75,4 +75,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index fd787ab..7f36c45 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -139,4 +139,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index b2f3fe1..bf27988 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -21,11 +21,11 @@
import java.util.Map;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.metadata.utils.MetadataConstants;
public class DatasetDecl implements Statement {
protected final Identifier name;
@@ -153,4 +153,9 @@
return dataverse;
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
index 89933e5..5067f9f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
@@ -44,4 +44,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
index 0054140..64ba087 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
@@ -51,4 +51,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
index ffe2c0d..d779b4f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
@@ -117,4 +117,9 @@
return equals && ObjectUtils.equals(dataverses, target.dataverses)
&& ObjectUtils.equals(rewrittenQuery, target.rewrittenQuery) && ObjectUtils.equals(vars, target.vars);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
index 9d8cb9d..af4d1fb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
@@ -75,4 +75,9 @@
return "disconnect feed " + feedName + " from " + datasetName;
}
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
similarity index 88%
rename from asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropStatement.java
rename to asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
index f4189dd..7dc1ad8 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
@@ -23,13 +23,13 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class DropStatement implements Statement {
+public class DropDatasetStatement implements Statement {
private final Identifier dataverseName;
private final Identifier datasetName;
private boolean ifExists;
- public DropStatement(Identifier dataverseName, Identifier datasetName, boolean ifExists) {
+ public DropDatasetStatement(Identifier dataverseName, Identifier datasetName, boolean ifExists) {
this.dataverseName = dataverseName;
this.datasetName = datasetName;
this.ifExists = ifExists;
@@ -57,4 +57,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
index 878fc49..0391a4b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
@@ -56,4 +56,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
index c217690..d08dc31 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
@@ -57,4 +57,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
index c71f7f2..08c90c5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
@@ -81,4 +81,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
index ae1a29b..00ae48c6 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
@@ -51,4 +51,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
index 1e3f6b0..1b8d0d5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
@@ -63,4 +63,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
index 1e69a68..b2dc72e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
@@ -82,4 +82,9 @@
&& ObjectUtils.equals(dataverseName, target.dataverseName) && ObjectUtils.equals(query, target.query);
}
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
index 849b0bd..145514f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
@@ -84,4 +84,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
index 710bedd..e27b447 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
@@ -51,4 +51,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
index 4232584..092e804 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
@@ -62,4 +62,9 @@
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException {
return visitor.visit(this, arg);
}
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
index 9be4830..fd6a088 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
@@ -121,4 +121,9 @@
&& ObjectUtils.equals(datasets, target.datasets) && ObjectUtils.equals(dataverses, target.dataverses)
&& topLevel == target.topLevel;
}
+
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
index 277b4bf..ab35b27 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
@@ -54,4 +54,9 @@
return Statement.Kind.EXTERNAL_DATASET_REFRESH;
}
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RunStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RunStatement.java
index 6b33078..006799d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RunStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RunStatement.java
@@ -78,4 +78,9 @@
return Statement.Kind.RUN;
}
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
index 54ba599..dfa8406 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
@@ -50,4 +50,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
index e25688e..a9eb795 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
@@ -76,4 +76,9 @@
return datagenAnnotation;
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
index d675054..99b9bb1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
@@ -57,4 +57,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
index 8ccc694..09ed242 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
@@ -86,4 +86,9 @@
&& ObjectUtils.equals(ucs, update.ucs) && ObjectUtils.equals(vars, update.vars);
}
+ @Override
+ public byte getCategory() {
+ return Category.UPDATE;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
index a755a54..311e650 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
@@ -57,4 +57,9 @@
return visitor.visit(this, arg);
}
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
+
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
new file mode 100644
index 0000000..94144c6
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -0,0 +1,168 @@
+/*
+ * 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.lang.common.util;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
+import org.apache.asterix.lang.common.expression.FieldBinding;
+import org.apache.asterix.lang.common.expression.ListConstructor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+@SuppressWarnings("unchecked")
+public class LangRecordParseUtil {
+ private static final String NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE =
+ "JSON record can only have expressions [%1$s, %2$s, %3$s]";
+ private static final ISerializerDeserializer<ADouble> doubleSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADOUBLE);
+ private static final ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+ private static final ISerializerDeserializer<AInt64> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ private static final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ private static final ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+
+ private LangRecordParseUtil() {
+ }
+
+ private static void parseExpression(Expression expr, ArrayBackedValueStorage serialized)
+ throws HyracksDataException {
+ switch (expr.getKind()) {
+ case LITERAL_EXPRESSION:
+ parseLiteral((LiteralExpr) expr, serialized);
+ break;
+ case RECORD_CONSTRUCTOR_EXPRESSION:
+ parseRecord((RecordConstructor) expr, serialized);
+ break;
+ case LIST_CONSTRUCTOR_EXPRESSION:
+ parseList((ListConstructor) expr, serialized);
+ break;
+ default:
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_PARSE_ERROR,
+ NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE,
+ new Serializable[] { Expression.Kind.LITERAL_EXPRESSION.toString(),
+ Expression.Kind.RECORD_CONSTRUCTOR_EXPRESSION.toString(),
+ Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION.toString() });
+ }
+ }
+
+ public static void parseRecord(RecordConstructor recordValue, ArrayBackedValueStorage serialized)
+ throws HyracksDataException {
+ AMutableString fieldNameString = new AMutableString(null);
+ ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ RecordBuilder recordBuilder = new RecordBuilder();
+ recordBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+ recordBuilder.init();
+ List<FieldBinding> fbList = recordValue.getFbList();
+ for (FieldBinding fb : fbList) {
+ fieldName.reset();
+ fieldValue.reset();
+ // get key
+ Expression keyExpr = fb.getLeftExpr();
+ if (keyExpr.getKind() != Expression.Kind.LITERAL_EXPRESSION) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_PARSE_ERROR,
+ "JSON key can only be of type %1$s", Expression.Kind.LITERAL_EXPRESSION);
+ }
+ LiteralExpr keyLiteralExpr = (LiteralExpr) keyExpr;
+ Literal keyLiteral = keyLiteralExpr.getValue();
+ if (keyLiteral.getLiteralType() != Literal.Type.STRING) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_PARSE_ERROR,
+ "JSON key can only be of type %1$s", Literal.Type.STRING);
+ }
+ fieldNameString.setValue(keyLiteral.getStringValue());
+ stringSerde.serialize(fieldNameString, fieldName.getDataOutput());
+ // get value
+ parseExpression(fb.getRightExpr(), fieldValue);
+ recordBuilder.addField(fieldName, fieldValue);
+ }
+ recordBuilder.write(serialized.getDataOutput(), true);
+ }
+
+ private static void parseList(ListConstructor valueExpr, ArrayBackedValueStorage serialized)
+ throws HyracksDataException {
+ if (valueExpr.getType() != ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR) {
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_PARSE_ERROR,
+ "JSON List can't be of type %1$s", valueExpr.getType());
+ }
+ ArrayBackedValueStorage serializedValue = new ArrayBackedValueStorage();
+ OrderedListBuilder listBuilder = new OrderedListBuilder();
+ listBuilder.reset(null);
+ for (Expression expr : valueExpr.getExprList()) {
+ serializedValue.reset();
+ parseExpression(expr, serializedValue);
+ listBuilder.addItem(serializedValue);
+ }
+ listBuilder.write(serialized.getDataOutput(), true);
+ }
+
+ private static void parseLiteral(LiteralExpr objectExpr, ArrayBackedValueStorage serialized)
+ throws HyracksDataException {
+ Literal value = objectExpr.getValue();
+ switch (value.getLiteralType()) {
+ case DOUBLE:
+ doubleSerde.serialize(new ADouble((Double) value.getValue()), serialized.getDataOutput());
+ break;
+ case TRUE:
+ booleanSerde.serialize(ABoolean.TRUE, serialized.getDataOutput());
+ break;
+ case FALSE:
+ booleanSerde.serialize(ABoolean.FALSE, serialized.getDataOutput());
+ break;
+ case FLOAT:
+ doubleSerde.serialize(new ADouble((Float) value.getValue()), serialized.getDataOutput());
+ break;
+ case INTEGER:
+ intSerde.serialize(new AInt64(((Integer) value.getValue()).longValue()), serialized.getDataOutput());
+ break;
+ case LONG:
+ intSerde.serialize(new AInt64((Long) value.getValue()), serialized.getDataOutput());
+ break;
+ case NULL:
+ nullSerde.serialize(ANull.NULL, serialized.getDataOutput());
+ break;
+ case STRING:
+ stringSerde.serialize(new AString((String) value.getValue()), serialized.getDataOutput());
+ break;
+ default:
+ throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_PARSE_ERROR,
+ "Unknown Literal Type %1$s", value.getLiteralType());
+ }
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 5feb61b..5ff4a03 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -30,7 +30,6 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.common.base.Expression;
@@ -74,7 +73,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
@@ -97,6 +96,7 @@
import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.UnaryExprType;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
@@ -566,7 +566,7 @@
}
@Override
- public Void visit(DropStatement del, Integer step) throws AsterixException {
+ public Void visit(DropDatasetStatement del, Integer step) throws AsterixException {
out.println(
skip(step) + "drop " + datasetSymbol + generateFullName(del.getDataverseName(), del.getDatasetName())
+ generateIfExists(del.getIfExists()) + SEMICOLON);
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 1e8dfd8..e32615f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -37,7 +37,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDropStatement;
@@ -70,7 +70,7 @@
}
@Override
- public R visit(DropStatement del, T arg) throws AsterixException {
+ public R visit(DropDatasetStatement del, T arg) throws AsterixException {
return null;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index 1545f3e..4a16025 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -53,7 +53,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDecl;
@@ -84,7 +84,7 @@
R visit(LoadStatement stmtLoad, T arg) throws AsterixException;
- R visit(DropStatement del, T arg) throws AsterixException;
+ R visit(DropDatasetStatement del, T arg) throws AsterixException;
R visit(InsertStatement insert, T arg) throws AsterixException;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
index a3346ed..fbf08ee 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
@@ -21,7 +21,6 @@
import java.util.ArrayList;
import java.util.List;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.common.base.Expression;
@@ -34,6 +33,7 @@
import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
public class SqlppBuiltinFunctionRewriteVisitor extends AbstractSqlppSimpleExpressionVisitor {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 9769d4f..e6c95b7 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -22,7 +22,6 @@
import java.util.List;
import java.util.Set;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.common.base.Expression;
@@ -39,6 +38,7 @@
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopingVisitor {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
index 9c3e1fb..6ae55dd 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
@@ -25,7 +25,6 @@
import java.util.Set;
import java.util.Map.Entry;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.common.base.Expression;
@@ -59,6 +58,7 @@
import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.core.algebra.base.Counter;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 95f09f5..2378060 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -45,7 +45,6 @@
import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.lang.common.base.Expression;
@@ -102,7 +101,7 @@
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DropStatement;
+import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
@@ -145,6 +144,7 @@
import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
import org.apache.asterix.lang.sqlpp.util.ExpressionToVariableUtil;
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
@@ -314,7 +314,7 @@
| stmt = InsertStatement()
| stmt = DeleteStatement()
| stmt = UpdateStatement()
- | stmt = FeedStatement()
+ | stmt = ConnectionStatement()
| stmt = CompactStatement()
| stmt = ExplainStatement()
| stmt = Query(false) <SEMICOLON>
@@ -870,7 +870,7 @@
(
Dataset() pairId = QualifiedName() ifExists = IfExists()
{
- stmt = new DropStatement(pairId.first, pairId.second, ifExists);
+ stmt = new DropDatasetStatement(pairId.first, pairId.second, ifExists);
}
| <INDEX> tripleId = DoubleQualifiedName() ifExists = IfExists()
{
@@ -1097,7 +1097,7 @@
}
}
-Statement FeedStatement() throws ParseException:
+Statement ConnectionStatement() throws ParseException:
{
Pair<Identifier,Identifier> feedNameComponents = null;
Pair<Identifier,Identifier> datasetNameComponents = null;
@@ -1108,18 +1108,54 @@
}
{
(
- <CONNECT> <FEED> feedNameComponents = QualifiedName() <TO> Dataset() datasetNameComponents = QualifiedName() (policy = GetPolicy())?
- {
- stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, policy, getVarCounter());
- }
- | <DISCONNECT> <FEED> feedNameComponents = QualifiedName() <FROM> Dataset() datasetNameComponents = QualifiedName()
+ <CONNECT> stmt = ConnectStatement()
+ | <DISCONNECT> stmt = DisconnectStatement()
+ )
+ {
+ return stmt;
+ }
+}
+
+Statement DisconnectStatement() throws ParseException:
+{
+ Pair<Identifier,Identifier> feedNameComponents = null;
+ Pair<Identifier,Identifier> datasetNameComponents = null;
+
+ Map<String,String> configuration = null;
+ Statement stmt = null;
+ String policy = null;
+}
+{
+ (
+ <FEED> feedNameComponents = QualifiedName() <FROM> Dataset() datasetNameComponents = QualifiedName()
{
stmt = new DisconnectFeedStatement(feedNameComponents, datasetNameComponents);
}
)
- {
- return stmt;
- }
+ {
+ return stmt;
+ }
+}
+
+Statement ConnectStatement() throws ParseException:
+{
+ Pair<Identifier,Identifier> feedNameComponents = null;
+ Pair<Identifier,Identifier> datasetNameComponents = null;
+
+ Map<String,String> configuration = null;
+ Statement stmt = null;
+ String policy = null;
+}
+{
+ (
+ <FEED> feedNameComponents = QualifiedName() <TO> Dataset() datasetNameComponents = QualifiedName() (policy = GetPolicy())?
+ {
+ stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, policy, getVarCounter());
+ }
+ )
+ {
+ return stmt;
+ }
}
Map<String,String> Configuration() throws ParseException :
@@ -2289,13 +2325,15 @@
}
}
-SelectExpression SelectExpression(boolean subquery) throws ParseException: {
+SelectExpression SelectExpression(boolean subquery) throws ParseException:
+{
List<LetClause> letClauses = new ArrayList<LetClause>();
SelectSetOperation selectSetOperation;
OrderbyClause orderbyClause = null;
LimitClause limitClause = null;
createNewScope();
-} {
+}
+{
( letClauses = LetClause() )?
selectSetOperation = SelectSetOperation()
(orderbyClause = OrderbyClause() {})?
@@ -2305,7 +2343,8 @@
}
}
-SelectSetOperation SelectSetOperation() throws ParseException: {
+SelectSetOperation SelectSetOperation() throws ParseException:
+{
SetOperationInput setOperationInputLeft;
List<SetOperationRight> setOperationRights = new ArrayList<SetOperationRight>();
}
@@ -2337,7 +2376,8 @@
}
}
-SelectExpression Subquery() throws ParseException: {
+SelectExpression Subquery() throws ParseException:
+{
SelectExpression selectExpr = null;
}
{
@@ -2347,7 +2387,8 @@
}
}
-SelectBlock SelectBlock() throws ParseException: {
+SelectBlock SelectBlock() throws ParseException:
+{
SelectClause selectClause = null;
FromClause fromClause = null;
List<LetClause> fromLetClauses = null;
@@ -2397,7 +2438,8 @@
}
}
-SelectClause SelectClause() throws ParseException: {
+SelectClause SelectClause() throws ParseException:
+{
SelectRegular selectRegular = null;
SelectElement selectElement = null;
boolean distinct = false;
@@ -2420,7 +2462,8 @@
}
}
-SelectRegular SelectRegular() throws ParseException: {
+SelectRegular SelectRegular() throws ParseException:
+{
List<Projection> projections = new ArrayList<Projection>();
}
{
@@ -2436,7 +2479,8 @@
}
}
-SelectElement SelectElement() throws ParseException: {
+SelectElement SelectElement() throws ParseException:
+{
Expression expr = null;
String name = null;
}
@@ -2447,7 +2491,8 @@
}
}
-Projection Projection() throws ParseException: {
+Projection Projection() throws ParseException :
+{
Expression expr = null;
Identifier identifier = null;
String name = null;
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
index d3e4a41..f66c449 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
@@ -1,102 +1,112 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-maven-plugins</artifactId>
- <version>0.8.9-SNAPSHOT</version>
- </parent>
- <artifactId>asterix-grammar-extension-maven-plugin</artifactId>
- <packaging>maven-plugin</packaging>
- <dependencies>
- <dependency>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-plugin-api</artifactId>
- <version>3.2.5</version>
- </dependency>
- <dependency>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-core</artifactId>
- <version>3.2.5</version>
- </dependency>
- <dependency>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-compat</artifactId>
- <version>3.2.5</version>
- </dependency>
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-maven-plugins</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-grammar-extension-maven-plugin</artifactId>
+ <packaging>maven-plugin</packaging>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-plugin-api</artifactId>
+ <version>3.2.5</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-core</artifactId>
+ <version>3.2.5</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-compat</artifactId>
+ <version>3.2.5</version>
+ </dependency>
<!-- dependencies to annotations -->
- <dependency>
- <groupId>org.apache.maven.plugin-tools</groupId>
- <artifactId>maven-plugin-annotations</artifactId>
- <version>3.4</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-external-data</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.maven.plugin-testing</groupId>
- <artifactId>maven-plugin-testing-harness</artifactId>
- <version>3.3.0</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-lang-common</artifactId>
- <version>${project.version}</version>
- </dependency>
- </dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>javacc-maven-plugin</artifactId>
- <version>2.6</version>
- <executions>
- <execution>
- <id>javacc</id>
- <goals>
- <goal>javacc</goal>
- </goals>
- <phase>verify</phase>
- <configuration>
- <isStatic>false</isStatic>
- <javaUnicodeEscape>true</javaUnicodeEscape>
- <sourceDirectory>${project.build.directory}/generated-sources/lang</sourceDirectory>
- <outputDirectory>${project.build.directory}/generated-sources/javacc</outputDirectory>
- </configuration>
- </execution>
- <execution>
- <id>javacc-jjdoc</id>
- <goals>
- <goal>jjdoc</goal>
- </goals>
- <phase>verify</phase>
- </execution>
- </executions>
- </plugin>
- </plugins>
- <pluginManagement>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-plugin-plugin</artifactId>
- <version>3.3</version>
- <configuration>
- <skipErrorNoDescriptorsFound>true</skipErrorNoDescriptorsFound>
- </configuration>
- <executions>
- <execution>
- <id>mojo-descriptor</id>
- <phase>process-classes</phase>
- <goals>
- <goal>descriptor</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </pluginManagement>
- </build>
+ <dependency>
+ <groupId>org.apache.maven.plugin-tools</groupId>
+ <artifactId>maven-plugin-annotations</artifactId>
+ <version>3.4</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-project</artifactId>
+ <version>2.0.8</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-external-data</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.plugin-testing</groupId>
+ <artifactId>maven-plugin-testing-harness</artifactId>
+ <version>3.3.0</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-lang-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-lang-sqlpp</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ </dependencies>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>javacc-maven-plugin</artifactId>
+ <version>2.6</version>
+ <executions>
+ <execution>
+ <id>javacc</id>
+ <goals>
+ <goal>javacc</goal>
+ </goals>
+ <phase>verify</phase>
+ <configuration>
+ <isStatic>false</isStatic>
+ <javaUnicodeEscape>true</javaUnicodeEscape>
+ <sourceDirectory>${project.build.directory}/generated-sources/lang</sourceDirectory>
+ <outputDirectory>${project.build.directory}/generated-sources/javacc</outputDirectory>
+ </configuration>
+ </execution>
+ <execution>
+ <id>javacc-jjdoc</id>
+ <goals>
+ <goal>jjdoc</goal>
+ </goals>
+ <phase>verify</phase>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ <pluginManagement>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-plugin-plugin</artifactId>
+ <version>3.3</version>
+ <configuration>
+ <skipErrorNoDescriptorsFound>true</skipErrorNoDescriptorsFound>
+ </configuration>
+ <executions>
+ <execution>
+ <id>mojo-descriptor</id>
+ <phase>process-classes</phase>
+ <goals>
+ <goal>descriptor</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </pluginManagement>
+ </build>
</project>
\ No newline at end of file
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java
index 8f263f7..5b44e23 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java
@@ -32,6 +32,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -60,20 +61,23 @@
private static final char OPEN_PAREN = '(';
private static final char CLOSE_PAREN = ')';
private static final char SEMICOLON = ';';
- private static final List<Character> SIG_SPECIAL_CHARS =
- Arrays.asList(new Character[] { '(', ')', ':', '<', '>', ';', '.' });
+ private static final List<Character> SIG_SPECIAL_CHARS = Arrays
+ .asList(new Character[] { '(', ')', ':', '<', '>', ';', '.' });
private static final String KWCLASS = "class";
private static final String KWIMPORT = "import";
+ private static final String KWUNIMPORT = "unimport";
private static final String KWPACKAGE = "package";
private static final String NEWPRODUCTION = "@new";
private static final String MERGEPRODUCTION = "@merge";
private static final String OVERRIDEPRODUCTION = "@override";
private static final String BEFORE = "before:";
private static final String AFTER = "after:";
- private static final List<String> KEYWORDS =
- Arrays.asList(new String[] { KWCLASS, KWIMPORT, KWPACKAGE, PARSER_BEGIN, PARSER_END });
- private static final List<String> EXTENSIONKEYWORDS =
- Arrays.asList(new String[] { KWIMPORT, NEWPRODUCTION, OVERRIDEPRODUCTION, MERGEPRODUCTION });
+ private static final String REPLACE = "replace";
+ private static final String WITH = "with";
+ private static final List<String> KEYWORDS = Arrays
+ .asList(new String[] { KWCLASS, KWIMPORT, KWPACKAGE, PARSER_BEGIN, PARSER_END });
+ private static final List<String> EXTENSIONKEYWORDS = Arrays
+ .asList(new String[] { KWIMPORT, KWUNIMPORT, NEWPRODUCTION, OVERRIDEPRODUCTION, MERGEPRODUCTION });
private static final String REGEX_WS_DOT_SEMICOLON = "\\s|[.]|[;]";
private static final String REGEX_WS_PAREN = "\\s|[(]|[)]";
private static final String OPTIONS = "options";
@@ -81,18 +85,25 @@
private Position position = new Position();
private Map<String, Pair<String, String>> extensibles = new HashMap<>();
private Map<String, String[]> mergeElements = new HashMap<>();
- private List<Pair<String, String>> finals = new ArrayList<>();
+ private List<Pair<String, String>> baseFinals = new ArrayList<>();
+ private List<Pair<String, String>> extensionFinals = new ArrayList<>();
private List<List<String>> imports = new ArrayList<>();
private String baseClassName;
private String baseClassDef;
private String optionsBlock;
private boolean read = false;
+ private boolean shouldReplace = false;
+ private String oldWord = null;
+ private String newWord = null;
@Parameter(property = "grammarix.base")
private String base;
- @Parameter(property = "grammarix.extension")
- private String extension;
+ @Parameter(property = "grammarix.gbase")
+ private String gbase;
+
+ @Parameter(property = "grammarix.gextension")
+ private String gextension;
@Parameter(property = "grammarix.output")
private String output;
@@ -106,9 +117,10 @@
@Override
public void execute() throws MojoExecutionException {
- getLog().info("Current dir: " + System.getProperty("user.dir"));
- getLog().info("base: " + base);
- getLog().info("extension: " + extension);
+ base = new File(base).getAbsolutePath();
+ getLog().info("Base dir: " + base);
+ getLog().info("Grammar-base: " + gbase);
+ getLog().info("Grammar-extension: " + gextension);
processBase();
processExtension();
generateOutput();
@@ -187,7 +199,14 @@
}
}
- for (Pair<String, String> element : finals) {
+ for (Pair<String, String> element : extensionFinals) {
+ writer.write(toOutput(element.first));
+ writer.newLine();
+ writer.write(element.second);
+ writer.newLine();
+ }
+
+ for (Pair<String, String> element : baseFinals) {
writer.write(toOutput(element.first));
writer.newLine();
writer.write(element.second);
@@ -247,8 +266,6 @@
// second block
writer.write(OPEN_BRACE);
writer.newLine();
- writer.write(" ");
- writer.write(OPEN_PAREN);
if (extensions[2] != null) {
writer.write(extensions[2]);
}
@@ -258,21 +275,33 @@
Position blockPosition = new Position();
blockPosition.index = 0;
blockPosition.line = blockReader.readLine();
- while (blockPosition.line != null && blockPosition.line.trim().length() == 0) {
+ while (blockPosition.line != null
+ && (blockPosition.line.trim().length() == 0 || blockPosition.line.indexOf(OPEN_BRACE) < 0)) {
+ if (blockPosition.line.trim().length() > 0) {
+ writer.write(blockPosition.line);
+ writer.newLine();
+ }
blockPosition.line = blockReader.readLine();
}
+ if (blockPosition.line == null) {
+ throw new MojoExecutionException(errorMessage);
+ }
int block2Open = blockPosition.line.indexOf(OPEN_BRACE);
- if (block2Open < 0) {
- throw new MojoExecutionException(errorMessage);
- }
blockPosition.line = blockPosition.line.substring(block2Open + 1);
- while (blockPosition.line != null && blockPosition.line.trim().length() == 0) {
+ while (blockPosition.line != null
+ && (blockPosition.line.trim().length() == 0 || blockPosition.line.indexOf(OPEN_PAREN) < 0)) {
+ if (blockPosition.line.trim().length() > 0) {
+ writer.write(blockPosition.line);
+ writer.newLine();
+ }
blockPosition.line = blockReader.readLine();
}
- int innerBlock1Open = blockPosition.line.indexOf(OPEN_PAREN);
- if (innerBlock1Open < 0) {
+ if (blockPosition.line == null) {
throw new MojoExecutionException(errorMessage);
}
+ int innerBlock1Open = blockPosition.line.indexOf(OPEN_PAREN);
+ writer.write(" ");
+ writer.write(OPEN_PAREN);
blockPosition.index = innerBlock1Open;
readBlock(blockReader, OPEN_PAREN, CLOSE_PAREN, blockPosition);
String innerBlock1String = record.toString();
@@ -397,7 +426,7 @@
}
private void processBase() throws MojoExecutionException {
- try (BufferedReader reader = Files.newBufferedReader(Paths.get(base), StandardCharsets.UTF_8)) {
+ try (BufferedReader reader = Files.newBufferedReader(Paths.get(base, gbase), StandardCharsets.UTF_8)) {
StringBuilder identifier = new StringBuilder();
while ((position.line = reader.readLine()) != null) {
if (position.line.trim().startsWith("//")) {
@@ -424,7 +453,7 @@
} else if (openAngularIndex == 0) {
position.index = position.line.indexOf(OPEN_ANGULAR);
readFinalProduction(identifier, reader);
- addFinalProduction(identifier);
+ addFinalProduction(identifier, baseFinals);
} else if (identifier.length() > 0 || position.line.trim().length() > 0) {
identifier.append(position.line);
identifier.append('\n');
@@ -463,7 +492,7 @@
}
}
- private void addFinalProduction(StringBuilder identifier) {
+ private void addFinalProduction(StringBuilder identifier, List<Pair<String, String>> finals) {
String sig = toSignature(identifier.toString());
finals.add(new Pair<String, String>(sig, record.toString()));
record.reset();
@@ -484,6 +513,34 @@
imports.add(importList);
}
+ private void handleUnImport(BufferedReader reader) throws IOException {
+ ArrayList<String> importList = new ArrayList<>();
+ String[] tokens = position.line.split(REGEX_WS_DOT_SEMICOLON);
+ importList.addAll(Arrays.asList(tokens));
+ while (position.line.indexOf(SEMICOLON) < 0) {
+ position.line = reader.readLine();
+ tokens = position.line.split(REGEX_WS_DOT_SEMICOLON);
+ importList.addAll(Arrays.asList(tokens));
+ }
+ // remove from imports
+ Iterator<List<String>> it = imports.iterator();
+ while (it.hasNext()) {
+ List<String> anImport = it.next();
+ if (anImport.size() == importList.size()) {
+ boolean equals = true;
+ for (int i = 1; i < anImport.size(); i++) {
+ if (!anImport.get(i).equals(importList.get(i))) {
+ equals = false;
+ break;
+ }
+ }
+ if (equals) {
+ it.remove();
+ }
+ }
+ }
+ }
+
private String importToString(List<String> importTokens) {
return "import " + StringUtils.join(importTokens.subList(1, importTokens.size()), '.') + ";";
}
@@ -590,7 +647,7 @@
}
private void processExtension() throws MojoExecutionException {
- try (BufferedReader reader = Files.newBufferedReader(Paths.get(extension), StandardCharsets.UTF_8)) {
+ try (BufferedReader reader = Files.newBufferedReader(Paths.get(base, gextension), StandardCharsets.UTF_8)) {
StringBuilder identifier = new StringBuilder();
String nextOperation = OVERRIDEPRODUCTION;
while (read || (position.line = reader.readLine()) != null) {
@@ -607,13 +664,16 @@
switch (tokens[0]) {
case KWIMPORT:
handleImport(reader);
- // import statement
+ break;
+ case KWUNIMPORT:
+ handleUnImport(reader);
break;
case NEWPRODUCTION:
nextOperation = NEWPRODUCTION;
break;
case MERGEPRODUCTION:
nextOperation = MERGEPRODUCTION;
+ shouldReplace = shouldReplace(tokens);
break;
case OVERRIDEPRODUCTION:
nextOperation = OVERRIDEPRODUCTION;
@@ -649,7 +709,7 @@
}
position.index = position.line.indexOf(OPEN_ANGULAR);
readFinalProduction(identifier, reader);
- addFinalProduction(identifier);
+ addFinalProduction(identifier, extensionFinals);
} else if (identifier.length() > 0 || position.line.trim().length() > 0) {
identifier.append(position.line);
identifier.append('\n');
@@ -661,6 +721,20 @@
}
}
+ private boolean shouldReplace(String[] tokens) throws MojoExecutionException {
+ boolean replace = false;
+ if (tokens.length == 5) {
+ if (tokens[1].equals(REPLACE) && tokens[3].equals(WITH)) {
+ shouldReplace = true;
+ oldWord = tokens[2];
+ newWord = tokens[4];
+ } else {
+ throw new MojoExecutionException("Allowed syntax after @merge: <REPLACE> oldWord <WITH> newWord");
+ }
+ }
+ return replace;
+ }
+
private void handleOverride(StringBuilder identifier, BufferedReader reader)
throws MojoExecutionException, IOException {
readBlock(reader, OPEN_BRACE, CLOSE_BRACE);
@@ -700,6 +774,11 @@
String sig = toSignature(identifier.toString());
if (!extensibles.containsKey(sig)) {
throw new MojoExecutionException(identifier.toString() + " doesn't exist in base grammar");
+ } else if (shouldReplace) {
+ Pair<String, String> baseMethods = extensibles.get(sig);
+ baseMethods.first = baseMethods.first.replaceAll(oldWord, newWord);
+ baseMethods.second = baseMethods.second.replaceAll(oldWord, newWord);
+ shouldReplace = false;
}
String[] amendments = new String[6];
mergeElements.put(sig, amendments);
@@ -769,8 +848,8 @@
int after = block.indexOf(AFTER);
if (before >= 0) {
// before exists
- amendments[beforeIndex] =
- block.substring(before + BEFORE.length(), (after >= 0) ? after : block.length() - 1);
+ amendments[beforeIndex] = block.substring(before + BEFORE.length(),
+ (after >= 0) ? after : block.length() - 1);
if (amendments[beforeIndex].trim().length() == 0) {
amendments[beforeIndex] = null;
}
@@ -791,7 +870,7 @@
private File prepareOutputFile() throws MojoExecutionException {
// write output
- File outputFile = new File(output);
+ File outputFile = new File(base, output);
if (outputFile.exists() && (!outputFile.delete())) {
throw new MojoExecutionException("Unable to delete file " + output);
}
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
index 7c6e882..3d98299 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
@@ -44,4 +44,9 @@
return arg;
}
+ @Override
+ public byte getCategory() {
+ return Category.PROCEDURE;
+ }
+
}
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/lang/extension.jj b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/lang/extension.jj
index 2106e4d..fb143a3 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/lang/extension.jj
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/lang/extension.jj
@@ -1,9 +1,14 @@
import org.apache.asterix.lang.extension.EchoStatement;
+// to remove an import, we use the keyword, unimport
+// unimport package name
// Merging of non-terminals can only be done on non-terminals which conform to the following structure.
// Content will simply be prepended or appended to the base blocks.
// Note: refrain from using the strings "before:" and "after:" in the merge areas as that will break the merge.
// As a workaround, you can always override
+// one additional possible change is direct replacement and it can be done through
+// @merge replace "baseWord" with "extensionWord"
+
@merge
Statement SingleStatement() throws ParseException:
{
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/unit/basic-test/basic-test-plugin-config.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/unit/basic-test/basic-test-plugin-config.xml
index b8253bb..5475fa4 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/unit/basic-test/basic-test-plugin-config.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/resources/unit/basic-test/basic-test-plugin-config.xml
@@ -1,80 +1,86 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-opt</artifactId>
+ <version>0.1-SNAPSHOT</version>
+ </parent>
+ <groupId>org.apache.extension</groupId>
+ <artifactId>grammar-extension</artifactId>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-external-data</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>com.couchbase.client</groupId>
+ <artifactId>core-io</artifactId>
+ <version>1.2.8</version>
+ </dependency>
+ <dependency>
+ <groupId>io.reactivex</groupId>
+ <artifactId>rxjava</artifactId>
+ <version>1.0.15</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-project</artifactId>
+ <version>2.0.8</version>
+ </dependency>
+ </dependencies>
+ <build>
+ <plugins>
+ <plugin>
<groupId>org.apache.asterix</groupId>
- <artifactId>asterix-opt</artifactId>
- <version>0.1-SNAPSHOT</version>
- </parent>
- <groupId>org.apache.extension</groupId>
- <artifactId>grammar-extension</artifactId>
- <dependencies>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-external-data</artifactId>
- <version>0.8.9-SNAPSHOT</version>
- </dependency>
- <dependency>
- <groupId>com.couchbase.client</groupId>
- <artifactId>core-io</artifactId>
- <version>1.2.8</version>
- </dependency>
- <dependency>
- <groupId>io.reactivex</groupId>
- <artifactId>rxjava</artifactId>
- <version>1.0.15</version>
- </dependency>
- </dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-grammar-extension-maven-plugin</artifactId>
- <version>0.8.9-SNAPSHOT</version>
- <configuration>
- <base>../../asterix-lang-aql/src/main/javacc/AQL.jj</base>
- <extension>src/test/resources/lang/extension.jj</extension>
- <output>target/generated-sources/lang/grammar.jj</output>
- <parserClassName>ExtendedParser</parserClassName>
- <packageName>org.apache.asterix.lang.extension.parser</packageName>
- </configuration>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals>
- <goal>grammarix</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- </plugins>
- <pluginManagement>
- <plugins>
+ <artifactId>asterix-grammar-extension-maven-plugin</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ <configuration>
+ <base>./</base>
+ <gbase>../../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj</gbase>
+ <gextension>src/test/resources/lang/extension.jj</gextension>
+ <output>target/generated-sources/lang/grammar.jj</output>
+ <parserClassName>ExtendedParser</parserClassName>
+ <packageName>org.apache.asterix.lang.extension.parser</packageName>
+ </configuration>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>grammarix</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ <pluginManagement>
+ <plugins>
<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
- <plugin>
- <groupId>org.eclipse.m2e</groupId>
- <artifactId>lifecycle-mapping</artifactId>
- <version>1.0.0</version>
- <configuration>
- <lifecycleMappingMetadata>
- <pluginExecutions>
- <pluginExecution>
- <pluginExecutionFilter>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-grammar-extension-maven-plugin</artifactId>
- <versionRange>[0.8.9-SNAPSHOT,)</versionRange>
- <goals>
- <goal>grammarix</goal>
- </goals>
- </pluginExecutionFilter>
- <action>
- <ignore></ignore>
- </action>
- </pluginExecution>
- </pluginExecutions>
- </lifecycleMappingMetadata>
- </configuration>
- </plugin>
- </plugins>
- </pluginManagement>
- </build>
+ <plugin>
+ <groupId>org.eclipse.m2e</groupId>
+ <artifactId>lifecycle-mapping</artifactId>
+ <version>1.0.0</version>
+ <configuration>
+ <lifecycleMappingMetadata>
+ <pluginExecutions>
+ <pluginExecution>
+ <pluginExecutionFilter>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-grammar-extension-maven-plugin</artifactId>
+ <versionRange>[0.8.9-SNAPSHOT,)</versionRange>
+ <goals>
+ <goal>grammarix</goal>
+ </goals>
+ </pluginExecutionFilter>
+ <action>
+ <ignore></ignore>
+ </action>
+ </pluginExecution>
+ </pluginExecutions>
+ </lifecycleMappingMetadata>
+ </configuration>
+ </plugin>
+ </plugins>
+ </pluginManagement>
+ </build>
</project>
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 9786ebf..70da097 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -31,6 +31,8 @@
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.metadata.api.IAsterixStateProxy;
+import org.apache.asterix.metadata.api.IExtensionMetadataEntity;
+import org.apache.asterix.metadata.api.IExtensionMetadataSearchKey;
import org.apache.asterix.metadata.api.IMetadataManager;
import org.apache.asterix.metadata.api.IMetadataNode;
import org.apache.asterix.metadata.entities.CompactionPolicy;
@@ -944,4 +946,38 @@
}
return dataset;
}
+
+ @Override
+ public <T extends IExtensionMetadataEntity> void addEntity(MetadataTransactionContext mdTxnCtx, T entity)
+ throws MetadataException {
+ try {
+ metadataNode.addEntity(mdTxnCtx.getJobId(), entity);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public <T extends IExtensionMetadataEntity> void deleteEntity(MetadataTransactionContext mdTxnCtx, T entity)
+ throws MetadataException {
+ try {
+ metadataNode.deleteEntity(mdTxnCtx.getJobId(), entity);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public <T extends IExtensionMetadataEntity> List<T> getEntities(MetadataTransactionContext mdTxnCtx,
+ IExtensionMetadataSearchKey searchKey) throws MetadataException {
+ try {
+ return metadataNode.getEntities(mdTxnCtx.getJobId(), searchKey);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ public static synchronized void instantiate(MetadataManager metadataManager) {
+ MetadataManager.INSTANCE = metadataManager;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 2cd8c46..383039e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -23,7 +23,9 @@
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
@@ -41,6 +43,12 @@
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.metadata.api.ExtensionMetadataDataset;
+import org.apache.asterix.metadata.api.ExtensionMetadataDatasetId;
+import org.apache.asterix.metadata.api.IExtensionMetadataEntity;
+import org.apache.asterix.metadata.api.IExtensionMetadataSearchKey;
+import org.apache.asterix.metadata.api.IMetadataEntityTupleTranslator;
+import org.apache.asterix.metadata.api.IMetadataExtension;
import org.apache.asterix.metadata.api.IMetadataIndex;
import org.apache.asterix.metadata.api.IMetadataNode;
import org.apache.asterix.metadata.api.IValueExtractor;
@@ -70,6 +78,7 @@
import org.apache.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
import org.apache.asterix.metadata.entitytupletranslators.NodeGroupTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.NodeTupleTranslator;
import org.apache.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
@@ -113,11 +122,16 @@
private static final long serialVersionUID = 1L;
private static final DatasetId METADATA_DATASET_ID = new DatasetId(
- MetadataIndexImmutableProperties.METADATA.getDatasetId());
+ MetadataPrimaryIndexes.PROPERTIES_METADATA.getDatasetId());
+ // shared between core and extension
private IDatasetLifecycleManager datasetLifecycleManager;
private ITransactionSubsystem transactionSubsystem;
private int metadataStoragePartition;
+ // core only
+ private transient MetadataTupleTranslatorProvider tupleTranslatorProvider;
+ // extension only
+ private Map<ExtensionMetadataDatasetId, ExtensionMetadataDataset<?>> extensionDatasets;
public static final MetadataNode INSTANCE = new MetadataNode();
@@ -125,11 +139,21 @@
super();
}
- public void initialize(IAsterixAppRuntimeContext runtimeContext) {
+ public void initialize(IAsterixAppRuntimeContext runtimeContext,
+ MetadataTupleTranslatorProvider tupleTranslatorProvider, List<IMetadataExtension> metadataExtensions) {
+ this.tupleTranslatorProvider = tupleTranslatorProvider;
this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
this.datasetLifecycleManager = runtimeContext.getDatasetLifecycleManager();
this.metadataStoragePartition = ((IAsterixPropertiesProvider) runtimeContext).getMetadataProperties()
.getMetadataPartition().getPartitionId();
+ if (metadataExtensions != null) {
+ extensionDatasets = new HashMap<>();
+ for (IMetadataExtension metadataExtension : metadataExtensions) {
+ for (ExtensionMetadataDataset<?> extensionIndex : metadataExtension.getExtensionIndexes()) {
+ extensionDatasets.put(extensionIndex.getId(), extensionIndex);
+ }
+ }
+ }
}
@Override
@@ -168,10 +192,112 @@
transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
}
+ // TODO(amoudi): make all metadata operations go through the generic methods
+ /**
+ * Add entity to index
+ *
+ * @param jobId
+ * @param entity
+ * @param tupleTranslator
+ * @param index
+ * @throws MetadataException
+ */
+ private <T> void addEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
+ IMetadataIndex index) throws MetadataException {
+ try {
+ ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
+ insertTupleIntoIndex(jobId, index, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException(entity.toString() + " already exists.", e);
+ } catch (ACIDException | IndexException | IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ /**
+ * Delete entity from index
+ *
+ * @param jobId
+ * @param entity
+ * @param tupleTranslator
+ * @param index
+ * @throws MetadataException
+ */
+ private <T> void deleteEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
+ IMetadataIndex index) throws MetadataException {
+ try {
+ ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
+ deleteTupleFromIndex(jobId, index, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException(entity.toString() + " already exists.", e);
+ } catch (ACIDException | IndexException | IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ /**
+ * retrieve all entities that matches the searchKey
+ *
+ * @param jobId
+ * @param searchKey
+ * @param tupleTranslator
+ * @param index
+ * @return
+ * @throws MetadataException
+ */
+ private <T> List<T> getEntities(JobId jobId, ITupleReference searchKey,
+ IMetadataEntityTupleTranslator<T> tupleTranslator, IMetadataIndex index) throws MetadataException {
+ try {
+ IValueExtractor<T> valueExtractor = new MetadataEntityValueExtractor<>(tupleTranslator);
+ List<T> results = new ArrayList<>();
+ searchIndex(jobId, index, searchKey, valueExtractor, results);
+ return results;
+ } catch (IndexException | IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <T extends IExtensionMetadataEntity> void addEntity(JobId jobId, T entity)
+ throws MetadataException, RemoteException {
+ ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
+ if (index == null) {
+ throw new MetadataException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
+ }
+ IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
+ addEntity(jobId, entity, tupleTranslator, index);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
+ throws MetadataException, RemoteException {
+ ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
+ if (index == null) {
+ throw new MetadataException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
+ }
+ IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
+ deleteEntity(jobId, entity, tupleTranslator, index);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
+ throws MetadataException, RemoteException {
+ ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets
+ .get(searchKey.getDatasetId());
+ if (index == null) {
+ throw new MetadataException("Metadata Extension Index: " + searchKey.getDatasetId() + " was not found");
+ }
+ IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
+ return getEntities(jobId, searchKey.getSearchKey(), tupleTranslator, index);
+ }
+
@Override
public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
try {
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
+ DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
} catch (TreeIndexDuplicateKeyException e) {
@@ -186,7 +312,7 @@
public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
try {
// Insert into the 'dataset' dataset.
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
+ DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(true);
ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
@@ -210,7 +336,7 @@
@Override
public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
try {
- IndexTupleTranslator tupleWriter = new IndexTupleTranslator(jobId, this, true);
+ IndexTupleTranslator tupleWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, true);
ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
} catch (TreeIndexDuplicateKeyException e) {
@@ -223,7 +349,7 @@
@Override
public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
try {
- NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
+ NodeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
} catch (TreeIndexDuplicateKeyException e) {
@@ -236,7 +362,7 @@
@Override
public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
try {
- NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
+ NodeGroupTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeGroupTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
} catch (TreeIndexDuplicateKeyException e) {
@@ -250,7 +376,8 @@
@Override
public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
try {
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
+ DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
+ true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
} catch (TreeIndexDuplicateKeyException e) {
@@ -264,7 +391,7 @@
public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
try {
// Insert into the 'function' dataset.
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
+ FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(true);
ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
@@ -572,9 +699,9 @@
@Override
public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
try {
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
- IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
- List<Dataverse> results = new ArrayList<Dataverse>();
+ DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(false);
+ IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Dataverse> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -586,9 +713,9 @@
public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
- IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
- List<Dataverse> results = new ArrayList<Dataverse>();
+ DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(false);
+ IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Dataverse> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -604,9 +731,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- List<Dataset> results = new ArrayList<Dataset>();
+ DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Dataset> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -618,9 +745,9 @@
public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(false);
- IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<Feed>(tupleReaderWriter);
- List<Feed> results = new ArrayList<Feed>();
+ FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
+ IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Feed> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -633,9 +760,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(false);
- IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<Library>(tupleReaderWriter);
- List<Library> results = new ArrayList<Library>();
+ LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(false);
+ IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Library> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -646,9 +773,10 @@
private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
+ DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
+ false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -661,9 +789,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- List<Dataset> results = new ArrayList<Dataset>();
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
+ List<Dataset> results = new ArrayList<>();
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -677,9 +805,9 @@
public List<Dataset> getAllDatasets(JobId jobId) throws MetadataException {
try {
ITupleReference searchKey = null;
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- List<Dataset> results = new ArrayList<Dataset>();
+ DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Dataset> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -690,9 +818,10 @@
public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException {
try {
ITupleReference searchKey = null;
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
+ DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
+ false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -785,7 +914,7 @@
public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
throws MetadataException {
//this needs to scan the datasets and return the datasets that use this nodegroup
- List<String> nodeGroupDatasets = new ArrayList<String>();
+ List<String> nodeGroupDatasets = new ArrayList<>();
List<Dataset> datasets = getAllDatasets(jobId);
for (Dataset set : datasets) {
if (set.getNodeGroupName().equals(nodegroup)) {
@@ -801,9 +930,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
- IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(jobId, this, false);
- IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
- List<Index> results = new ArrayList<Index>();
+ IndexTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this,
+ false);
+ IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Index> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -819,9 +949,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName);
- IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(jobId, this, false);
- IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
- List<Index> results = new ArrayList<Index>();
+ IndexTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this,
+ false);
+ IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Index> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -834,9 +965,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
+ DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
+ false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -851,9 +983,9 @@
public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(nodeGroupName);
- NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
- IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
- List<NodeGroup> results = new ArrayList<NodeGroup>();
+ NodeGroupTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeGroupTupleTranslator(false);
+ IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<NodeGroup> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -870,9 +1002,9 @@
try {
ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(),
"" + functionSignature.getArity());
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
- List<Function> results = new ArrayList<Function>();
- IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
+ FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
+ List<Function> results = new ArrayList<>();
+ IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -916,7 +1048,7 @@
private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
throws MetadataException, IndexException, IOException {
IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
- List<ITupleReference> results = new ArrayList<ITupleReference>();
+ List<ITupleReference> results = new ArrayList<>();
searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
if (results.isEmpty()) {
// TODO: Temporarily a TreeIndexException to make it get caught by
@@ -1059,8 +1191,8 @@
NoOpOperationCallback.INSTANCE);
IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
indexAccessor.search(rangeCursor, rangePred);
@@ -1092,7 +1224,7 @@
// TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
// Hyracks version.
- public ITupleReference createTuple(String... fields) throws HyracksDataException {
+ public static ITupleReference createTuple(String... fields) {
@SuppressWarnings("unchecked")
ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ASTRING);
@@ -1100,7 +1232,12 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fields.length);
for (String s : fields) {
aString.setValue(s);
- stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ try {
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ } catch (HyracksDataException e) {
+ // This should never happen
+ throw new IllegalStateException("Failed to create search tuple!!!! This should never happen", e);
+ }
tupleBuilder.addFieldEndOffset();
}
ArrayTupleReference tuple = new ArrayTupleReference();
@@ -1113,9 +1250,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
- IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
- List<Function> results = new ArrayList<Function>();
+ FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
+ IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<Function> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -1127,7 +1264,8 @@
public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
try {
// Insert into the 'Adapter' dataset.
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
+ DatasourceAdapterTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getAdapterTupleTranslator(true);
ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
@@ -1172,10 +1310,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, adapterName);
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
- tupleReaderWriter);
+ DatasourceAdapterTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getAdapterTupleTranslator(false);
+ List<DatasourceAdapter> results = new ArrayList<>();
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -1191,7 +1329,8 @@
throws MetadataException, RemoteException {
try {
// Insert into the 'CompactionPolicy' dataset.
- CompactionPolicyTupleTranslator tupleReaderWriter = new CompactionPolicyTupleTranslator(true);
+ CompactionPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getCompactionPolicyTupleTranslator(true);
ITupleReference compactionPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(compactionPolicy);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, compactionPolicyTuple);
@@ -1208,10 +1347,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, policyName);
- CompactionPolicyTupleTranslator tupleReaderWriter = new CompactionPolicyTupleTranslator(false);
- List<CompactionPolicy> results = new ArrayList<CompactionPolicy>();
- IValueExtractor<CompactionPolicy> valueExtractor = new MetadataEntityValueExtractor<CompactionPolicy>(
- tupleReaderWriter);
+ CompactionPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getCompactionPolicyTupleTranslator(false);
+ List<CompactionPolicy> results = new ArrayList<>();
+ IValueExtractor<CompactionPolicy> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, searchKey, valueExtractor, results);
if (!results.isEmpty()) {
return results.get(0);
@@ -1227,10 +1366,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
- tupleReaderWriter);
- List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
+ DatasourceAdapterTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getAdapterTupleTranslator(false);
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<DatasourceAdapter> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -1242,7 +1381,7 @@
public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException {
try {
// Insert into the 'Library' dataset.
- LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(true);
+ LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(true);
ITupleReference libraryTuple = tupleReaderWriter.getTupleFromMetadataEntity(library);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, libraryTuple);
@@ -1285,9 +1424,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, libraryName);
- LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(false);
- List<Library> results = new ArrayList<Library>();
- IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<Library>(tupleReaderWriter);
+ LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(false);
+ List<Library> results = new ArrayList<>();
+ IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -1307,7 +1446,7 @@
public void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws MetadataException, RemoteException {
try {
// Insert into the 'FeedPolicy' dataset.
- FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(true);
+ FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(true);
ITupleReference feedPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedPolicy);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
@@ -1324,10 +1463,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, policyName);
- FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(false);
- List<FeedPolicyEntity> results = new ArrayList<FeedPolicyEntity>();
- IValueExtractor<FeedPolicyEntity> valueExtractor = new MetadataEntityValueExtractor<FeedPolicyEntity>(
- tupleReaderWriter);
+ FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(false);
+ List<FeedPolicyEntity> results = new ArrayList<>();
+ IValueExtractor<FeedPolicyEntity> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
if (!results.isEmpty()) {
return results.get(0);
@@ -1342,7 +1480,7 @@
public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException {
try {
// Insert into the 'Feed' dataset.
- FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(true);
+ FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(true);
ITupleReference feedTuple = tupleReaderWriter.getTupleFromMetadataEntity(feed);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
@@ -1358,9 +1496,9 @@
public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, feedName);
- FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(false);
- List<Feed> results = new ArrayList<Feed>();
- IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<Feed>(tupleReaderWriter);
+ FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
+ List<Feed> results = new ArrayList<>();
+ IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
if (!results.isEmpty()) {
return results.get(0);
@@ -1407,10 +1545,9 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse);
- FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(false);
- IValueExtractor<FeedPolicyEntity> valueExtractor = new MetadataEntityValueExtractor<FeedPolicyEntity>(
- tupleReaderWriter);
- List<FeedPolicyEntity> results = new ArrayList<FeedPolicyEntity>();
+ FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(false);
+ IValueExtractor<FeedPolicyEntity> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<FeedPolicyEntity> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -1422,7 +1559,8 @@
public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException {
try {
// Insert into the 'externalFiles' dataset.
- ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(true);
+ ExternalFileTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getExternalFileTupleTranslator(true);
ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
} catch (TreeIndexDuplicateKeyException e) {
@@ -1438,10 +1576,10 @@
public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
- ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
- IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
- tupleReaderWriter);
- List<ExternalFile> results = new ArrayList<ExternalFile>();
+ ExternalFileTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getExternalFileTupleTranslator(false);
+ IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<ExternalFile> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (IndexException | IOException e) {
@@ -1513,10 +1651,10 @@
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
- ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
- IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
- tupleReaderWriter);
- List<ExternalFile> results = new ArrayList<ExternalFile>();
+ ExternalFileTupleTranslator tupleReaderWriter = tupleTranslatorProvider
+ .getExternalFileTupleTranslator(false);
+ IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<ExternalFile> results = new ArrayList<>();
searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
@@ -1541,7 +1679,7 @@
deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
// Previous tuple was deleted
// Insert into the 'dataset' dataset.
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
+ DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(true);
datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
} catch (ACIDException | IndexException | IOException e) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java
new file mode 100644
index 0000000..d3f7e1c
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java
@@ -0,0 +1,51 @@
+/*
+ * 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.metadata.api;
+
+import java.util.List;
+
+import org.apache.asterix.metadata.bootstrap.MetadataIndex;
+import org.apache.asterix.metadata.bootstrap.MetadataIndexImmutableProperties;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+
+public class ExtensionMetadataDataset<T> extends MetadataIndex {
+
+ private static final long serialVersionUID = 1L;
+ private final ExtensionMetadataDatasetId indexId;
+ private final transient IMetadataEntityTupleTranslator<T> tupleTranslator;
+
+ public ExtensionMetadataDataset(MetadataIndexImmutableProperties indexProperties, int numFields, IAType[] keyTypes,
+ List<List<String>> keyNames, int numSecondaryIndexKeys, ARecordType payloadType, boolean isPrimaryIndex,
+ int[] primaryKeyIndexes, ExtensionMetadataDatasetId indexId,
+ IMetadataEntityTupleTranslator<T> tupleTranslator) {
+ super(indexProperties, numFields, keyTypes, keyNames, numSecondaryIndexKeys, payloadType, isPrimaryIndex,
+ primaryKeyIndexes);
+ this.indexId = indexId;
+ this.tupleTranslator = tupleTranslator;
+ }
+
+ public ExtensionMetadataDatasetId getId() {
+ return indexId;
+ }
+
+ public IMetadataEntityTupleTranslator<T> getTupleTranslator() {
+ return tupleTranslator;
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDatasetId.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDatasetId.java
new file mode 100644
index 0000000..4074cdc
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDatasetId.java
@@ -0,0 +1,62 @@
+/*
+ * 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.metadata.api;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.api.ExtensionId;
+import org.apache.commons.lang3.ObjectUtils;
+
+public class ExtensionMetadataDatasetId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private final ExtensionId extensionId;
+ private final String datasetName;
+
+ public ExtensionMetadataDatasetId(ExtensionId extensionId, String datasetName) {
+ this.extensionId = extensionId;
+ this.datasetName = datasetName;
+ }
+
+ public ExtensionId getExtensionId() {
+ return extensionId;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ }
+ if (o instanceof ExtensionMetadataDatasetId) {
+ ExtensionMetadataDatasetId otherId = (ExtensionMetadataDatasetId) o;
+ return ObjectUtils.equals(extensionId, otherId.getExtensionId())
+ && ObjectUtils.equals(datasetName, otherId.getDatasetName());
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return ObjectUtils.hashCodeMulti(datasetName, extensionId);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IExtensionMetadataEntity.java
similarity index 69%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
copy to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IExtensionMetadataEntity.java
index 1752054..6ae23f0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IExtensionMetadataEntity.java
@@ -16,12 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.metadata.api;
-import java.util.Map;
+import java.io.Serializable;
-public interface ISubscriberRuntime {
+/**
+ * Represents an extension metadata entity
+ */
+public interface IExtensionMetadataEntity extends Serializable {
- public Map<String, String> getFeedPolicy();
-
+ /**
+ * Get the dataset Id of the dataset of the entity
+ *
+ * @return the dataset Id
+ */
+ ExtensionMetadataDatasetId getDatasetId();
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IExtensionMetadataSearchKey.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IExtensionMetadataSearchKey.java
new file mode 100644
index 0000000..e8e39e3
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IExtensionMetadataSearchKey.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.metadata.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * A search key used to search extension datasets for extension entities
+ */
+public interface IExtensionMetadataSearchKey extends Serializable {
+
+ /**
+ * Gets the dataset id for the extension metadata dataset to be searched
+ *
+ * @return the dataset Id
+ */
+ ExtensionMetadataDatasetId getDatasetId();
+
+ /**
+ * Gets the tuple reference representation of the search key
+ *
+ * @return the search key as a tuple reference
+ */
+ ITupleReference getSearchKey();
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java
index 2fc846b..1759c0f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java
@@ -16,7 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.asterix.metadata.api;
import java.io.Serializable;
@@ -24,7 +23,6 @@
import org.apache.asterix.metadata.MetadataCache;
public interface IMetadataEntity<T> extends Serializable {
-
public static final int PENDING_NO_OP = 0;
public static final int PENDING_ADD_OP = 1;
public static final int PENDING_DROP_OP = 2;
@@ -32,4 +30,5 @@
public T addToCache(MetadataCache cache);
public T dropFromCache(MetadataCache cache);
+
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
index 7c115b8..d8ccbde 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
@@ -20,6 +20,7 @@
package org.apache.asterix.metadata.api;
import java.io.IOException;
+import java.io.Serializable;
import org.apache.asterix.metadata.MetadataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -30,7 +31,7 @@
* representation in a Hyracks tuple, and vice versa. Implementations of this
* interface are intended to be used within an IMetadataNode.
*/
-public interface IMetadataEntityTupleTranslator<T> {
+public interface IMetadataEntityTupleTranslator<T> extends Serializable {
/**
* Transforms a metadata entity of type T from a given tuple to a Java
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java
new file mode 100644
index 0000000..9b55151
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java
@@ -0,0 +1,61 @@
+/*
+ * 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.metadata.api;
+
+import java.rmi.RemoteException;
+import java.util.List;
+
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * An interface for Metadata Extensions
+ */
+public interface IMetadataExtension extends IExtension {
+
+ @Override
+ default ExtensionKind getExtensionKind() {
+ return ExtensionKind.METADATA;
+ }
+
+ /**
+ * @return The tuple translator provider that must be used by the {@code IMetadataNode } to read and write core
+ * {@code IMetadataEntity} objects
+ */
+ MetadataTupleTranslatorProvider getMetadataTupleTranslatorProvider();
+
+ /**
+ * @return A list of additional extension instances of {@code IMetadataIndex} that are introduced by the extension
+ */
+ @SuppressWarnings("rawtypes")
+ List<ExtensionMetadataDataset> getExtensionIndexes();
+
+ /**
+ * Called when booting the {@code IMetadataNode}
+ *
+ * @throws HyracksDataException
+ * @throws RemoteException
+ * @throws ACIDException
+ */
+ void initializeMetadata() throws HyracksDataException, RemoteException, ACIDException;
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataIndex.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataIndex.java
index 9b8bcfb..9eec64c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataIndex.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataIndex.java
@@ -19,6 +19,7 @@
package org.apache.asterix.metadata.api;
+import java.io.Serializable;
import java.util.List;
import org.apache.asterix.common.transactions.DatasetId;
@@ -33,7 +34,7 @@
/**
* Descriptor interface for a primary or secondary index on metadata datasets.
*/
-public interface IMetadataIndex {
+public interface IMetadataIndex extends Serializable {
public String getDataverseName();
public String getNodeGroupName();
@@ -83,5 +84,4 @@
boolean isPrimaryIndex();
int[] getPrimaryKeyIndexes();
-
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 242bbe3..0acc027 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -39,6 +39,7 @@
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.metadata.entities.Node;
import org.apache.asterix.metadata.entities.NodeGroup;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -56,41 +57,46 @@
/**
* Initializes the metadata manager, e.g., finds the remote metadata node.
+ *
* @throws RemoteException
* If an error occurred while contacting the proxy for finding
* the metadata node.
*/
- public void init() throws RemoteException, MetadataException;
+ void init() throws RemoteException, MetadataException;
/**
* Begins a transaction on the metadata node.
+ *
* @return A globally unique transaction id.
* @throws ACIDException
* @throws RemoteException
*/
- public MetadataTransactionContext beginTransaction() throws ACIDException, RemoteException;
+ MetadataTransactionContext beginTransaction() throws ACIDException, RemoteException;
/**
* Commits a remote transaction on the metadata node.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @throws ACIDException
* @throws RemoteException
*/
- public void commitTransaction(MetadataTransactionContext ctx) throws ACIDException, RemoteException;
+ void commitTransaction(MetadataTransactionContext ctx) throws ACIDException, RemoteException;
/**
* Aborts a remote transaction running on the metadata node.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @throws ACIDException
* @throws RemoteException
*/
- public void abortTransaction(MetadataTransactionContext ctx) throws ACIDException, RemoteException;
+ void abortTransaction(MetadataTransactionContext ctx) throws ACIDException, RemoteException;
/**
* Locks the metadata in given mode. The lock acquisition is delegated to
* the metadata node. This method blocks until the lock can be acquired.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param lockMode
@@ -98,19 +104,21 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void lock(MetadataTransactionContext ctx, byte lockMode) throws ACIDException, RemoteException;
+ void lock(MetadataTransactionContext ctx, byte lockMode) throws ACIDException, RemoteException;
/**
* Releases all locks on the metadata held by the given transaction id.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @throws ACIDException
* @throws RemoteException
*/
- public void unlock(MetadataTransactionContext ctx, byte lockMode) throws ACIDException, RemoteException;
+ void unlock(MetadataTransactionContext ctx, byte lockMode) throws ACIDException, RemoteException;
/**
* Inserts a new dataverse into the metadata.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverse
@@ -118,10 +126,11 @@
* @throws MetadataException
* For example, if the dataverse already exists.
*/
- public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException;
+ void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException;
/**
* Retrieves all dataverses
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @return A list of dataverse instances.
@@ -131,6 +140,7 @@
/**
* Retrieves a dataverse with given name.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -139,10 +149,11 @@
* @throws MetadataException
* For example, if the dataverse does not exist.
*/
- public Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
/**
* Retrieves all datasets belonging to the given dataverse.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -151,22 +162,23 @@
* @throws MetadataException
* For example, if the dataverse does not exist.
*/
- public List<Dataset> getDataverseDatasets(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException;
+ List<Dataset> getDataverseDatasets(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
/**
* Deletes the dataverse with given name, and all it's associated datasets,
* indexes, and types.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @return A list of dataset instances.
* @throws MetadataException
* For example, if the dataverse does not exist.
*/
- public void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
/**
* Inserts a new dataset into the metadata.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataset
@@ -174,10 +186,11 @@
* @throws MetadataException
* For example, if the dataset already exists.
*/
- public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
+ void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
/**
* Retrieves a dataset within a given dataverse.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -188,11 +201,12 @@
* @throws MetadataException
* For example, if the dataset does not exist.
*/
- public Dataset getDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
+ Dataset getDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
throws MetadataException;
/**
* Retrieves all indexes of a dataset.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -203,11 +217,12 @@
* @throws MetadataException
* For example, if the dataset and/or dataverse does not exist.
*/
- public List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
+ List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
throws MetadataException;
/**
* Deletes the dataset with given name, and all it's associated indexes.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -217,12 +232,12 @@
* @throws MetadataException
* For example, if the dataset and/or dataverse does not exist.
*/
- public void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
- throws MetadataException;
+ void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName) throws MetadataException;
/**
* Inserts an index into the metadata. The index itself knows its name, and
* which dataset it belongs to.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param index
@@ -230,10 +245,11 @@
* @throws MetadataException
* For example, if the index already exists.
*/
- public void addIndex(MetadataTransactionContext ctx, Index index) throws MetadataException;
+ void addIndex(MetadataTransactionContext ctx, Index index) throws MetadataException;
/**
* Retrieves the index with given name, in given dataverse and dataset.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -245,11 +261,12 @@
* @throws MetadataException
* For example, if the index does not exist.
*/
- public Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
+ Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
throws MetadataException;
/**
* Deletes the index with given name, in given dataverse and dataset.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -260,11 +277,12 @@
* @throws MetadataException
* For example, if the index does not exist.
*/
- public void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
+ void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
throws MetadataException;
/**
* Inserts a datatype.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param datatype
@@ -272,10 +290,11 @@
* @throws MetadataException
* For example, if the datatype already exists.
*/
- public void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws MetadataException;
+ void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws MetadataException;
/**
* Retrieves the datatype with given name in given dataverse.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -286,11 +305,12 @@
* @throws MetadataException
* For example, if the datatype does not exist.
*/
- public Datatype getDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
+ Datatype getDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
throws MetadataException;
/**
* Deletes the given datatype in given dataverse.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -301,11 +321,12 @@
* For example, if there are still datasets using the type to be
* deleted.
*/
- public void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
+ void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
throws MetadataException;
/**
* Inserts a node group.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param nodeGroup
@@ -313,10 +334,11 @@
* @throws MetadataException
* For example, if the node group already exists.
*/
- public void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws MetadataException;
+ void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws MetadataException;
/**
* Retrieves a node group.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param nodeGroupName
@@ -324,10 +346,11 @@
* @throws MetadataException
* For example, if the node group does not exist.
*/
- public NodeGroup getNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException;
+ NodeGroup getNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException;
/**
* Deletes a node group.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param nodeGroupName
@@ -336,10 +359,11 @@
* For example, there are still datasets partitioned on the node
* group to be deleted.
*/
- public void dropNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException;
+ void dropNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException;
/**
* Inserts a node (machine).
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param node
@@ -347,7 +371,7 @@
* @throws MetadataException
* For example, if the node already exists.
*/
- public void addNode(MetadataTransactionContext ctx, Node node) throws MetadataException;
+ void addNode(MetadataTransactionContext ctx, Node node) throws MetadataException;
/**
* @param mdTxnCtx
@@ -357,7 +381,7 @@
* being added
* @throws MetadataException
*/
- public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException;
+ void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException;
/**
* @param ctx
@@ -368,8 +392,7 @@
* @throws MetadataException
*/
- public Function getFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
- throws MetadataException;
+ Function getFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature) throws MetadataException;
/**
* @param ctx
@@ -378,8 +401,7 @@
* the functions signature (unique to the function)
* @throws MetadataException
*/
- public void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
- throws MetadataException;
+ void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature) throws MetadataException;
/**
* @param mdTxnCtx
@@ -389,7 +411,7 @@
* added
* @throws MetadataException
*/
- public void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws MetadataException;
+ void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws MetadataException;
/**
* @param ctx
@@ -401,7 +423,7 @@
* @return
* @throws MetadataException
*/
- public DatasourceAdapter getAdapter(MetadataTransactionContext ctx, String dataverseName, String name)
+ DatasourceAdapter getAdapter(MetadataTransactionContext ctx, String dataverseName, String name)
throws MetadataException;
/**
@@ -413,14 +435,14 @@
* name of the adapter
* @throws MetadataException
*/
- public void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws MetadataException;
+ void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws MetadataException;
/**
* @param ctx
* @param policy
* @throws MetadataException
*/
- public void addCompactionPolicy(MetadataTransactionContext ctx, CompactionPolicy policy) throws MetadataException;
+ void addCompactionPolicy(MetadataTransactionContext ctx, CompactionPolicy policy) throws MetadataException;
/**
* @param ctx
@@ -429,7 +451,7 @@
* @return
* @throws MetadataException
*/
- public CompactionPolicy getCompactionPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
+ CompactionPolicy getCompactionPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
throws MetadataException;
/**
@@ -438,15 +460,14 @@
* @return
* @throws MetadataException
*/
- public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException;
+ List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
/**
* @param ctx
* @param feed
* @throws MetadataException
*/
- public void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException;
+ void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException;
/**
* @param ctx
@@ -455,7 +476,7 @@
* @return
* @throws MetadataException
*/
- public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+ Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
/**
* @param ctx
@@ -463,14 +484,14 @@
* @param feedName
* @throws MetadataException
*/
- public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+ void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
/**
* @param ctx
* @param policy
* @throws MetadataException
*/
- public void addFeedPolicy(MetadataTransactionContext ctx, FeedPolicyEntity policy) throws MetadataException;
+ void addFeedPolicy(MetadataTransactionContext ctx, FeedPolicyEntity policy) throws MetadataException;
/**
* @param ctx
@@ -479,24 +500,25 @@
* @return
* @throws MetadataException
*/
- public FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
+ FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
throws MetadataException;
- public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
+ void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
- public int getMostRecentDatasetId() throws MetadataException;
+ int getMostRecentDatasetId() throws MetadataException;
- public void acquireWriteLatch();
+ void acquireWriteLatch();
- public void releaseWriteLatch();
+ void releaseWriteLatch();
- public void acquireReadLatch();
+ void acquireReadLatch();
- public void releaseReadLatch();
+ void releaseReadLatch();
/**
* Removes a library , acquiring local locks on behalf of the given
* transaction id.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -506,12 +528,12 @@
* if the library does not exists.
* @throws RemoteException
*/
- public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
- throws MetadataException;
+ void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName) throws MetadataException;
/**
* Adds a library, acquiring local locks on behalf of the given
* transaction id.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param library
@@ -520,7 +542,7 @@
* for example, if the library is already added.
* @throws RemoteException
*/
- public void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException;
+ void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException;
/**
* @param ctx
@@ -533,11 +555,12 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+ Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
throws MetadataException, RemoteException;
/**
* Retireve libraries installed in a given dataverse.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
@@ -546,8 +569,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException;
+ List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
/**
* @param mdTxnCtx
@@ -557,8 +579,7 @@
* added
* @throws MetadataException
*/
- public void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile)
- throws MetadataException;
+ void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
/**
* @param mdTxnCtx
@@ -568,7 +589,7 @@
* @return A list of external files belonging to the dataset
* @throws MetadataException
*/
- public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
+ List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
throws MetadataException;
/**
@@ -579,8 +600,7 @@
* dropped
* @throws MetadataException
*/
- public void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile)
- throws MetadataException;
+ void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
/**
* @param mdTxnCtx
@@ -590,10 +610,11 @@
* dropped
* @throws MetadataException
*/
- public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException;
+ void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException;
/**
* Get en external file
+ *
* @param mdTxnCtx
* @param dataverseName
* @param datasetName
@@ -601,11 +622,12 @@
* @return
* @throws MetadataException
*/
- public ExternalFile getExternalFile(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName,
+ ExternalFile getExternalFile(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName,
Integer fileNumber) throws MetadataException;
/**
* update an existing dataset in metadata.
+ *
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataset
@@ -613,12 +635,45 @@
* @throws MetadataException
* For example, if the dataset already exists.
*/
- public void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
+ void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
/**
* Clean up temporary datasets that have not been active for a long time.
+ *
* @throws MetadataException
*/
- public void cleanupTempDatasets() throws MetadataException;
+ void cleanupTempDatasets() throws MetadataException;
+
+ /**
+ * Add an extension entity to its extension dataset under the ongoing metadata transaction
+ *
+ * @param mdTxnCtx
+ * @param entity
+ * @throws MetadataException
+ */
+ <T extends IExtensionMetadataEntity> void addEntity(MetadataTransactionContext mdTxnCtx, T entity)
+ throws MetadataException;
+
+ /**
+ * Deletes an extension entity from its extension dataset under the ongoing metadata transaction
+ *
+ * @param mdTxnCtx
+ * @param entity
+ * @throws MetadataException
+ */
+ <T extends IExtensionMetadataEntity> void deleteEntity(MetadataTransactionContext mdTxnCtx, T entity)
+ throws MetadataException;
+
+ /**
+ * Gets a list of extension entities matching a search key under the ongoing metadata transaction
+ *
+ * @param mdTxnCtx
+ * @param searchKey
+ * @return
+ * @throws MetadataException
+ * @throws HyracksDataException
+ */
+ <T extends IExtensionMetadataEntity> List<T> getEntities(MetadataTransactionContext mdTxnCtx,
+ IExtensionMetadataSearchKey searchKey) throws MetadataException, HyracksDataException;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index ec2e692..41d0b6a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -59,7 +59,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void beginTransaction(JobId jobId) throws ACIDException, RemoteException;
+ void beginTransaction(JobId jobId) throws ACIDException, RemoteException;
/**
* Commits a local transaction against the metadata.
@@ -67,7 +67,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void commitTransaction(JobId jobId) throws ACIDException, RemoteException;
+ void commitTransaction(JobId jobId) throws ACIDException, RemoteException;
/**
* Aborts a local transaction against the metadata.
@@ -75,7 +75,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void abortTransaction(JobId jobId) throws ACIDException, RemoteException;
+ void abortTransaction(JobId jobId) throws ACIDException, RemoteException;
/**
* Locally locks the entire metadata in given mode on behalf of given
@@ -84,7 +84,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
+ void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
/**
* Releases all local locks of given transaction id.
@@ -92,7 +92,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void unlock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
+ void unlock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
/**
* Inserts a new dataverse into the metadata, acquiring local locks on
@@ -106,7 +106,7 @@
* For example, if the dataverse already exists.
* @throws RemoteException
*/
- public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException;
+ void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException;
/**
* Retrieves all dataverses, acquiring local locks on behalf of
@@ -119,7 +119,7 @@
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException;
+ List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException;
/**
* Retrieves a dataverse with given name, acquiring local locks on behalf of
@@ -134,7 +134,7 @@
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* Retrieves all datasets belonging to the given dataverse, acquiring local
@@ -148,8 +148,7 @@
* @throws MetadataException
* For example, if the dataverse does not exist. RemoteException
*/
- public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException;
+ List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* Deletes the dataverse with given name, and all it's associated datasets,
@@ -163,7 +162,7 @@
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* Inserts a new dataset into the metadata, acquiring local locks on behalf
@@ -177,7 +176,7 @@
* For example, if the dataset already exists.
* @throws RemoteException
*/
- public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
/**
* Retrieves a dataset within a given dataverse, acquiring local locks on
@@ -194,8 +193,7 @@
* For example, if the dataset does not exist.
* @throws RemoteException
*/
- public Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException;
+ Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException, RemoteException;
/**
* Retrieves all indexes of a dataset, acquiring local locks on behalf of
@@ -212,7 +210,7 @@
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
- public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
+ List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
throws MetadataException, RemoteException;
/**
@@ -229,8 +227,7 @@
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
- public void dropDataset(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException;
+ void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException, RemoteException;
/**
* Inserts an index into the metadata, acquiring local locks on behalf of
@@ -245,7 +242,7 @@
* For example, if the index already exists.
* @throws RemoteException
*/
- public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException;
+ void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException;
/**
* Retrieves the index with given name, in given dataverse and dataset,
@@ -263,7 +260,7 @@
* For example, if the index does not exist.
* @throws RemoteException
*/
- public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+ Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
throws MetadataException, RemoteException;
/**
@@ -281,7 +278,7 @@
* For example, if the index does not exist.
* @throws RemoteException
*/
- public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+ void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
throws MetadataException, RemoteException;
/**
@@ -296,7 +293,7 @@
* For example, if the datatype already exists.
* @throws RemoteException
*/
- public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException;
+ void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException;
/**
* Retrieves the datatype with given name in given dataverse, acquiring
@@ -313,7 +310,7 @@
* For example, if the datatype does not exist.
* @throws RemoteException
*/
- public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
+ Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
throws MetadataException, RemoteException;
/**
@@ -331,8 +328,7 @@
* deleted.
* @throws RemoteException
*/
- public void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException;
+ void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException, RemoteException;
/**
* Inserts a node group, acquiring local locks on behalf of the given
@@ -346,7 +342,7 @@
* For example, if the node group already exists.
* @throws RemoteException
*/
- public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
+ void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
/**
* Retrieves a node group, acquiring local locks on behalf of the given
@@ -360,7 +356,7 @@
* For example, if the node group does not exist.
* @throws RemoteException
*/
- public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
+ NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
/**
* Deletes a node group, acquiring local locks on behalf of the given
@@ -375,7 +371,7 @@
* group to be deleted.
* @throws RemoteException
*/
- public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
+ void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
/**
* Inserts a node (compute node), acquiring local locks on behalf of the
@@ -389,7 +385,7 @@
* For example, if the node already exists.
* @throws RemoteException
*/
- public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException;
+ void addNode(JobId jobId, Node node) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -400,8 +396,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Function getFunction(JobId jobId, FunctionSignature functionSignature)
- throws MetadataException, RemoteException;
+ Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
/**
* Deletes a function, acquiring local locks on behalf of the given
@@ -416,8 +411,7 @@
* group to be deleted.
* @throws RemoteException
*/
- public void dropFunction(JobId jobId, FunctionSignature functionSignature)
- throws MetadataException, RemoteException;
+ void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -429,7 +423,7 @@
* unknown function
* @throws RemoteException
*/
- public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException;
+ void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException;
/**
* @param ctx
@@ -438,8 +432,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Function> getDataverseFunctions(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException;
+ List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* @param ctx
@@ -448,7 +441,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
+ List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
throws MetadataException, RemoteException;
/**
@@ -459,7 +452,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+ DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
throws MetadataException, RemoteException;
/**
@@ -475,8 +468,7 @@
* if the adapter does not exists.
* @throws RemoteException
*/
- public void dropAdapter(JobId jobId, String dataverseName, String adapterName)
- throws MetadataException, RemoteException;
+ void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -487,7 +479,7 @@
* for example, if the adapter already exists.
* @throws RemoteException
*/
- public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
+ void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -495,8 +487,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
- throws MetadataException, RemoteException;
+ void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -506,7 +497,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy)
+ CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy)
throws MetadataException, RemoteException;
/**
@@ -514,14 +505,14 @@
* @throws MetadataException
* @throws RemoteException
*/
- public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
+ void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
/**
* @return
* @throws MetadataException
* @throws RemoteException
*/
- public int getMostRecentDatasetId() throws MetadataException, RemoteException;
+ int getMostRecentDatasetId() throws MetadataException, RemoteException;
/**
* @param jobId
@@ -529,7 +520,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException;
+ void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -539,7 +530,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+ Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -548,7 +539,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+ void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -556,7 +547,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws MetadataException, RemoteException;
+ void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -566,7 +557,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy)
+ FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy)
throws MetadataException, RemoteException;
/**
@@ -582,8 +573,7 @@
* if the library does not exists.
* @throws RemoteException
*/
- public void dropLibrary(JobId jobId, String dataverseName, String libraryName)
- throws MetadataException, RemoteException;
+ void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException, RemoteException;
/**
* Adds a library, acquiring local locks on behalf of the given
@@ -597,7 +587,7 @@
* for example, if the library is already added.
* @throws RemoteException
*/
- public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException;
+ void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException;
/**
* @param txnId
@@ -610,8 +600,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Library getLibrary(JobId jobId, String dataverseName, String libraryName)
- throws MetadataException, RemoteException;
+ Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException, RemoteException;
/**
* Retireve libraries installed in a given dataverse.
@@ -624,8 +613,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Library> getDataverseLibraries(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException;
+ List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -634,7 +622,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* delete a give feed (ingestion) policy
@@ -646,8 +634,7 @@
* @throws RemoteException
* @throws MetadataException
*/
- public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
- throws MetadataException, RemoteException;
+ void dropFeedPolicy(JobId jobId, String dataverseName, String policyName) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -656,7 +643,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
+ List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
throws MetadataException, RemoteException;
/**
@@ -668,7 +655,7 @@
* for example, if the file already exists.
* @throws RemoteException
*/
- public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException;
+ void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException;
/**
* @param jobId
@@ -678,7 +665,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
/**
* Deletes an externalFile , acquiring local locks on behalf of the given
@@ -694,7 +681,7 @@
* the id number for the file to be deleted
* @throws RemoteException
*/
- public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
+ void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
throws MetadataException, RemoteException;
/**
@@ -707,7 +694,7 @@
* An external dataset the files belong to.
* @throws RemoteException
*/
- public void dropExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ void dropExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
/**
* Retrieves the file with given number, in given dataverse and dataset,
@@ -726,7 +713,7 @@
* For example, if the index does not exist.
* @throws RemoteException
*/
- public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
+ ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
throws MetadataException, RemoteException;
/**
@@ -741,6 +728,40 @@
* For example, if the dataset already exists.
* @throws RemoteException
*/
- public void updateDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ void updateDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+
+ /**
+ * Adds an extension entity under the ongoing transaction job id
+ *
+ * @param jobId
+ * @param entity
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ <T extends IExtensionMetadataEntity> void addEntity(JobId jobId, T entity)
+ throws MetadataException, RemoteException;
+
+ /**
+ * Deletes an extension entity under the ongoing transaction job id
+ *
+ * @param jobId
+ * @param entity
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ <T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
+ throws MetadataException, RemoteException;
+
+ /**
+ * Gets a list of extension entities matching a search key under the ongoing transaction
+ *
+ * @param jobId
+ * @param searchKey
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
+ throws MetadataException, RemoteException;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 757694d..f5ccb9e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -35,7 +35,6 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.config.IAsterixPropertiesProvider;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.context.BaseOperationTracker;
import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
@@ -65,6 +64,7 @@
import org.apache.asterix.metadata.entities.Node;
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.util.AsterixClusterProperties;
@@ -75,6 +75,7 @@
import org.apache.hyracks.api.application.INCApplicationContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.storage.am.common.frames.LIFOMetaDataFrame;
@@ -103,47 +104,39 @@
* stopUniverse() should be called upon application undeployment.
*/
public class MetadataBootstrap {
- private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
public static final boolean IS_DEBUG_MODE = false;// true
-
+ private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
private static IAsterixAppRuntimeContext runtimeContext;
-
private static IBufferCache bufferCache;
private static IFileMapProvider fileMapProvider;
private static IDatasetLifecycleManager dataLifecycleManager;
private static ILocalResourceRepository localResourceRepository;
private static IIOManager ioManager;
-
private static String metadataNodeName;
private static List<String> nodeNames;
private static String outputDir;
+ private static boolean isNewUniverse;
- private static IMetadataIndex[] primaryIndexes;
+ private static final IMetadataIndex[] PRIMARY_INDEXES =
+ new IMetadataIndex[] { MetadataPrimaryIndexes.DATAVERSE_DATASET, MetadataPrimaryIndexes.DATASET_DATASET,
+ MetadataPrimaryIndexes.DATATYPE_DATASET, MetadataPrimaryIndexes.INDEX_DATASET,
+ MetadataPrimaryIndexes.NODE_DATASET, MetadataPrimaryIndexes.NODEGROUP_DATASET,
+ MetadataPrimaryIndexes.FUNCTION_DATASET, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET,
+ MetadataPrimaryIndexes.FEED_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
+ MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
+ MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
private static IAsterixPropertiesProvider propertiesProvider;
- private static void initLocalIndexArrays() {
- primaryIndexes = new IMetadataIndex[] { MetadataPrimaryIndexes.DATAVERSE_DATASET,
- MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
- MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
- MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
- MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
- MetadataPrimaryIndexes.FEED_POLICY_DATASET, MetadataPrimaryIndexes.LIBRARY_DATASET,
- MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
- }
-
public static void startUniverse(IAsterixPropertiesProvider asterixPropertiesProvider,
INCApplicationContext ncApplicationContext, boolean isNewUniverse) throws Exception {
+ MetadataBootstrap.setNewUniverse(isNewUniverse);
runtimeContext = (IAsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
propertiesProvider = asterixPropertiesProvider;
- MetadataPrimaryIndexes.init();
- initLocalIndexArrays();
-
AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
metadataNodeName = metadataProperties.getMetadataNodeName();
nodeNames = metadataProperties.getNodeNames();
-
dataLifecycleManager = runtimeContext.getDatasetLifecycleManager();
localResourceRepository = runtimeContext.getLocalResourceRepository();
bufferCache = runtimeContext.getBufferCache();
@@ -156,37 +149,26 @@
// Lock the metadata in X mode.
MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
+ for (int i = 0; i < PRIMARY_INDEXES.length; i++) {
+ enlistMetadataDataset(PRIMARY_INDEXES[i]);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(
+ "Finished enlistment of metadata B-trees in " + (isNewUniverse ? "new" : "old") + " universe");
+ }
if (isNewUniverse) {
- for (int i = 0; i < primaryIndexes.length; i++) {
- enlistMetadataDataset(primaryIndexes[i], true, mdTxnCtx);
- }
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Finished enlistment of metadata B-trees in new universe");
- }
-
insertInitialDataverses(mdTxnCtx);
- insertInitialDatasets(mdTxnCtx);
- insertInitialDatatypes(mdTxnCtx);
+ insertMetadataDatasets(mdTxnCtx, PRIMARY_INDEXES);
+ insertMetadataDatatypes(mdTxnCtx);
insertNodes(mdTxnCtx);
insertInitialGroups(mdTxnCtx);
insertInitialAdapters(mdTxnCtx);
insertInitialFeedPolicies(mdTxnCtx);
insertInitialCompactionPolicies(mdTxnCtx);
-
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Finished creating metadata B-trees.");
}
- } else {
- for (int i = 0; i < primaryIndexes.length; i++) {
- enlistMetadataDataset(primaryIndexes[i], false, mdTxnCtx);
- }
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Finished enlistment of metadata B-trees in old universe.");
- }
}
-
// #. initialize datasetIdFactory
MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -211,24 +193,26 @@
// metadata datasets will be closed when the dataset life cycle manger is closed
}
- public static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws Exception {
+ private static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws Exception {
String dataverseName = MetadataPrimaryIndexes.DATAVERSE_DATASET.getDataverseName();
String dataFormat = NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT;
MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
new Dataverse(dataverseName, dataFormat, IMetadataEntity.PENDING_NO_OP));
}
- public static void insertInitialDatasets(MetadataTransactionContext mdTxnCtx) throws Exception {
- for (int i = 0; i < primaryIndexes.length; i++) {
+ public static void insertMetadataDatasets(MetadataTransactionContext mdTxnCtx, IMetadataIndex[] indexes)
+ throws Exception {
+ for (int i = 0; i < indexes.length; i++) {
IDatasetDetails id = new InternalDatasetDetails(FileStructure.BTREE, PartitioningStrategy.HASH,
- primaryIndexes[i].getPartitioningExpr(), primaryIndexes[i].getPartitioningExpr(), null,
- primaryIndexes[i].getPartitioningExprType(), false, null, false);
- MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(primaryIndexes[i].getDataverseName(),
- primaryIndexes[i].getIndexedDatasetName(), primaryIndexes[i].getDataverseName(),
- primaryIndexes[i].getPayloadRecordType().getTypeName(), primaryIndexes[i].getNodeGroupName(),
- GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME, GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, id,
- new HashMap<String, String>(), DatasetType.INTERNAL, primaryIndexes[i].getDatasetId().getId(),
- IMetadataEntity.PENDING_NO_OP));
+ indexes[i].getPartitioningExpr(), indexes[i].getPartitioningExpr(), null,
+ indexes[i].getPartitioningExprType(), false, null, false);
+ MetadataManager.INSTANCE.addDataset(mdTxnCtx,
+ new Dataset(indexes[i].getDataverseName(), indexes[i].getIndexedDatasetName(),
+ indexes[i].getDataverseName(), indexes[i].getPayloadRecordType().getTypeName(),
+ indexes[i].getNodeGroupName(), GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME,
+ GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, id, new HashMap<String, String>(),
+ DatasetType.INTERNAL, indexes[i].getDatasetId().getId(),
+ IMetadataEntity.PENDING_NO_OP));
}
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Finished inserting initial datasets.");
@@ -244,12 +228,12 @@
}
public static void getMetadataTypes(ArrayList<IAType> types) throws Exception {
- for (int i = 0; i < primaryIndexes.length; i++) {
- types.add(primaryIndexes[i].getPayloadRecordType());
+ for (int i = 0; i < PRIMARY_INDEXES.length; i++) {
+ types.add(PRIMARY_INDEXES[i].getPayloadRecordType());
}
}
- public static void insertInitialDatatypes(MetadataTransactionContext mdTxnCtx) throws Exception {
+ public static void insertMetadataDatatypes(MetadataTransactionContext mdTxnCtx) throws Exception {
String dataverseName = MetadataPrimaryIndexes.DATAVERSE_DATASET.getDataverseName();
ArrayList<IAType> types = new ArrayList<IAType>();
getBuiltinTypes(types);
@@ -313,9 +297,9 @@
}
private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx) throws Exception {
- String[] builtInCompactionPolicyClassNames = new String[] { ConstantMergePolicyFactory.class.getName(),
- PrefixMergePolicyFactory.class.getName(), NoMergePolicyFactory.class.getName(),
- CorrelatedPrefixMergePolicyFactory.class.getName() };
+ String[] builtInCompactionPolicyClassNames =
+ new String[] { ConstantMergePolicyFactory.class.getName(), PrefixMergePolicyFactory.class.getName(),
+ NoMergePolicyFactory.class.getName(), CorrelatedPrefixMergePolicyFactory.class.getName() };
CompactionPolicy compactionPolicy;
for (String policyClassName : builtInCompactionPolicyClassNames) {
compactionPolicy = getCompactionPolicyEntity(policyClassName);
@@ -330,13 +314,12 @@
}
private static CompactionPolicy getCompactionPolicyEntity(String compactionPolicyClassName) throws Exception {
- String policyName = ((ILSMMergePolicyFactory) (Class.forName(compactionPolicyClassName).newInstance()))
- .getName();
+ String policyName =
+ ((ILSMMergePolicyFactory) (Class.forName(compactionPolicyClassName).newInstance())).getName();
return new CompactionPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, policyName, compactionPolicyClassName);
}
- private static void enlistMetadataDataset(IMetadataIndex index, boolean create, MetadataTransactionContext mdTxnCtx)
- throws Exception {
+ public static void enlistMetadataDataset(IMetadataIndex index) throws HyracksDataException {
ClusterPartition metadataPartition = propertiesProvider.getMetadataProperties().getMetadataPartition();
int metadataDeviceId = metadataPartition.getIODeviceNum();
String metadataPartitionPath = StoragePathUtil.prepareStoragePartitionPath(
@@ -349,14 +332,14 @@
ITypeTraits[] typeTraits = index.getTypeTraits();
IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
- LSMBTree lsmBtree = null;
- long resourceID = -1;
- ILSMOperationTracker opTracker = index.isPrimaryIndex()
- ? runtimeContext.getLSMBTreeOperationTracker(index.getDatasetId().getId())
- : new BaseOperationTracker(index.getDatasetId().getId(),
- dataLifecycleManager.getDatasetInfo(index.getDatasetId().getId()));
+ LSMBTree lsmBtree;
+ long resourceID;
+ ILSMOperationTracker opTracker =
+ index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index.getDatasetId().getId())
+ : new BaseOperationTracker(index.getDatasetId().getId(),
+ dataLifecycleManager.getDatasetInfo(index.getDatasetId().getId()));
final String absolutePath = file.getFile().getPath();
- if (create) {
+ if (isNewUniverse()) {
lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, runtimeContext.getBloomFilterFalsePositiveRate(),
runtimeContext.getMetadataMergePolicyFactory()
@@ -370,8 +353,8 @@
comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(), index.getDatasetId().getId(),
runtimeContext.getMetadataMergePolicyFactory(), GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES,
null, null, null, null);
- ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
- localResourceMetadata, LocalResource.LSMBTreeResource);
+ ILocalResourceFactoryProvider localResourceFactoryProvider =
+ new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.LSMBTreeResource);
ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, resourceName,
metadataPartition.getPartitionId(), LIFOMetaDataFrame.VERSION, absolutePath));
@@ -379,7 +362,7 @@
} else {
final LocalResource resource = localResourceRepository.getResourceByPath(absolutePath);
if (resource == null) {
- throw new Exception("Could not find required metadata indexes. Please delete "
+ throw new HyracksDataException("Could not find required metadata indexes. Please delete "
+ propertiesProvider.getMetadataProperties().getTransactionLogDirs()
.get(runtimeContext.getTransactionSubsystem().getId())
+ " to intialize as a new instance. (WARNING: all data will be lost.)");
@@ -398,7 +381,6 @@
dataLifecycleManager.register(absolutePath, lsmBtree);
}
}
-
index.setResourceID(resourceID);
index.setFile(file);
}
@@ -418,9 +400,7 @@
String datasetName = null;
String indexName = null;
MetadataTransactionContext mdTxnCtx = null;
-
MetadataManager.INSTANCE.acquireWriteLatch();
-
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting DDL recovery ...");
}
@@ -448,8 +428,8 @@
LOGGER.info("Dropped a pending dataset: " + dataverseName + "." + datasetName);
}
} else {
- List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
- datasetName);
+ List<Index> indexes =
+ MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
for (Index index : indexes) {
indexName = index.getIndexName();
if (index.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
@@ -464,11 +444,11 @@
}
if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
// if the dataset has no indexes, delete all its files
- List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
- datasetName);
+ List<Index> indexes =
+ MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
if (indexes.size() == 0) {
- List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx,
- dataset);
+ List<ExternalFile> files =
+ MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
for (ExternalFile file : files) {
MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
if (LOGGER.isLoggable(Level.INFO)) {
@@ -500,4 +480,12 @@
MetadataManager.INSTANCE.releaseWriteLatch();
}
}
+
+ public static boolean isNewUniverse() {
+ return isNewUniverse;
+ }
+
+ public static void setNewUniverse(boolean isNewUniverse) {
+ MetadataBootstrap.isNewUniverse = isNewUniverse;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndex.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndex.java
index 0a3d622..6c8bebd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndex.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndex.java
@@ -24,14 +24,13 @@
import java.util.Arrays;
import java.util.List;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.transactions.DatasetId;
import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.AqlBinaryHashFunctionFactoryProvider;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.formats.nontagged.AqlTypeTraitProvider;
-import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.api.IMetadataIndex;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -44,7 +43,9 @@
/**
* Descriptor for a primary or secondary index on metadata datasets.
*/
-public final class MetadataIndex implements IMetadataIndex {
+public class MetadataIndex implements IMetadataIndex {
+
+ private static final long serialVersionUID = 1L;
// Name of dataset that is indexed.
protected final String datasetName;
// Name of index. null for primary indexes. non-null for secondary indexes.
@@ -52,6 +53,7 @@
// Types of key fields.
protected final IAType[] keyTypes;
// Names of key fields. Used to compute partitionExprs.
+ // Note: used list implementation must implement java.io.Serializable
protected final List<List<String>> keyNames;
// Field permutation for BTree insert. Auto-created based on numFields.
protected final int[] fieldPermutation;
@@ -82,13 +84,13 @@
public MetadataIndex(MetadataIndexImmutableProperties indexImmutableProperties, int numFields, IAType[] keyTypes,
List<List<String>> keyNames, int numSecondaryIndexKeys, ARecordType payloadType, boolean isPrimaryIndex,
- int[] primaryKeyIndexes) throws MetadataException {
+ int[] primaryKeyIndexes) {
// Sanity checks.
if (keyTypes.length != keyNames.size()) {
- throw new MetadataException("Unequal number of key types and names given.");
+ throw new AssertionError("Unequal number of key names and key types");
}
if (keyTypes.length > numFields) {
- throw new MetadataException("Number of keys given is greater than total number of fields.");
+ throw new AssertionError("Key size is larger than total number of fields");
}
// Set simple fields.
this.datasetName = indexImmutableProperties.getDatasetName();
@@ -101,7 +103,7 @@
for (int i = 0; i < numFields; i++) {
fieldPermutation[i] = i;
}
- // Create serdes for RecordDescriptor;
+ // Create serdes for RecordDescriptor
@SuppressWarnings("rawtypes")
ISerializerDeserializer[] serdes = new ISerializerDeserializer[numFields];
for (int i = 0; i < keyTypes.length; i++) {
@@ -189,7 +191,7 @@
@Override
public List<List<String>> getPartitioningExpr() {
- ArrayList<List<String>> partitioningExpr = new ArrayList<List<String>>();
+ ArrayList<List<String>> partitioningExpr = new ArrayList<>();
for (int i = 0; i < keyNames.size(); i++) {
partitioningExpr.add(keyNames.get(i));
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndexImmutableProperties.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndexImmutableProperties.java
index 2c2468c..53a3cbd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndexImmutableProperties.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataIndexImmutableProperties.java
@@ -18,46 +18,19 @@
*/
package org.apache.asterix.metadata.bootstrap;
-import org.apache.asterix.common.config.MetadataConstants;
-
-public enum MetadataIndexImmutableProperties {
- METADATA(MetadataConstants.METADATA_DATAVERSE_NAME, 0, 0),
- DATAVERSE("Dataverse", 1, 1),
- DATASET("Dataset", 2, 2),
- DATATYPE("Datatype", 3, 3),
- INDEX("Index", 4, 4),
- NODE("Node", 5, 5),
- NODEGROUP("Nodegroup", 6, 6),
- FUNCTION("Function", 7, 7),
- DATASOURCE_ADAPTER("DatasourceAdapter", 8, 8),
- LIBRARY("Library", 9, 9),
- FEED("Feed", 10, 10),
- FEED_ACTIVITY_DATASET_ID("FeedActivity", 11, 11),
- FEED_POLICY("FeedPolicy", 12, 12),
- COMPACTION_POLICY("CompactionPolicy", 13, 13),
- EXTERNAL_FILE("ExternalFile", 14, 14);
-
+public class MetadataIndexImmutableProperties {
private final String indexName;
private final int datasetId;
private final long resourceId;
- private final MetadataIndexImmutableProperties dataset;
+ // TODO(till? should we reconsider these numbers?!)
+ public static final int FIRST_AVAILABLE_EXTENSION_METADATA_DATASET_ID = 52;
public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
- private MetadataIndexImmutableProperties(String indexName, int datasetId, long resourceId) {
+ public MetadataIndexImmutableProperties(String indexName, int datasetId, long resourceId) {
this.indexName = indexName;
this.datasetId = datasetId;
this.resourceId = resourceId;
- //a primary index's dataset is itself
- this.dataset = this;
- }
-
- private MetadataIndexImmutableProperties(String indexName, MetadataIndexImmutableProperties dataset,
- long resourceId) {
- this.indexName = indexName;
- this.datasetId = dataset.datasetId;
- this.resourceId = resourceId;
- this.dataset = dataset;
}
public long getResourceId() {
@@ -68,11 +41,12 @@
return indexName;
}
- public String getDatasetName() {
- return dataset.indexName;
+ public int getDatasetId() {
+ return datasetId;
}
- public int getDatasetId() {
- return dataset.datasetId;
+ // Right now, we only have primary indexes. Hence, dataset name is always index name
+ public String getDatasetName() {
+ return indexName;
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index d387041..8201627 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -21,8 +21,8 @@
import java.util.Arrays;
-import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.api.IMetadataIndex;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
@@ -30,99 +30,104 @@
* Contains static primary-index descriptors of all metadata datasets.
*/
public class MetadataPrimaryIndexes {
+ public static final MetadataIndexImmutableProperties PROPERTIES_METADATA =
+ new MetadataIndexImmutableProperties(MetadataConstants.METADATA_DATAVERSE_NAME, 0, 0);
+ public static final MetadataIndexImmutableProperties PROPERTIES_DATAVERSE =
+ new MetadataIndexImmutableProperties("Dataverse", 1, 1);
+ public static final MetadataIndexImmutableProperties PROPERTIES_DATASET =
+ new MetadataIndexImmutableProperties("Dataset", 2, 2);
+ public static final MetadataIndexImmutableProperties PROPERTIES_DATATYPE =
+ new MetadataIndexImmutableProperties("Datatype", 3, 3);
+ public static final MetadataIndexImmutableProperties PROPERTIES_INDEX =
+ new MetadataIndexImmutableProperties("Index", 4, 4);
+ public static final MetadataIndexImmutableProperties PROPERTIES_NODE =
+ new MetadataIndexImmutableProperties("Node", 5, 5);
+ public static final MetadataIndexImmutableProperties PROPERTIES_NODEGROUP =
+ new MetadataIndexImmutableProperties("Nodegroup", 6, 6);
+ public static final MetadataIndexImmutableProperties PROPERTIES_FUNCTION =
+ new MetadataIndexImmutableProperties("Function", 7, 7);
+ public static final MetadataIndexImmutableProperties PROPERTIES_DATASOURCE_ADAPTER =
+ new MetadataIndexImmutableProperties("DatasourceAdapter", 8, 8);
+ public static final MetadataIndexImmutableProperties PROPERTIES_LIBRARY =
+ new MetadataIndexImmutableProperties("Library", 9, 9);
+ public static final MetadataIndexImmutableProperties PROPERTIES_FEED =
+ new MetadataIndexImmutableProperties("Feed", 10, 10);
+ public static final MetadataIndexImmutableProperties PROPERTIES_FEED_ACTIVITY_DATASET_ID =
+ new MetadataIndexImmutableProperties("FeedActivity", 11, 11);
+ public static final MetadataIndexImmutableProperties PROPERTIES_FEED_POLICY =
+ new MetadataIndexImmutableProperties("FeedPolicy", 12, 12);
+ public static final MetadataIndexImmutableProperties PROPERTIES_COMPACTION_POLICY =
+ new MetadataIndexImmutableProperties("CompactionPolicy", 13, 13);
+ public static final MetadataIndexImmutableProperties PROPERTIES_EXTERNAL_FILE =
+ new MetadataIndexImmutableProperties("ExternalFile", 14, 14);
- public static IMetadataIndex DATAVERSE_DATASET;
- public static IMetadataIndex DATASET_DATASET;
- public static IMetadataIndex DATATYPE_DATASET;
- public static IMetadataIndex INDEX_DATASET;
- public static IMetadataIndex NODE_DATASET;
- public static IMetadataIndex NODEGROUP_DATASET;
- public static IMetadataIndex FUNCTION_DATASET;
- public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
- public static IMetadataIndex LIBRARY_DATASET;
- public static IMetadataIndex FEED_DATASET;
- public static IMetadataIndex FEED_ACTIVITY_DATASET;
- public static IMetadataIndex FEED_POLICY_DATASET;
- public static IMetadataIndex COMPACTION_POLICY_DATASET;
- public static IMetadataIndex EXTERNAL_FILE_DATASET;
+ public static final IMetadataIndex DATAVERSE_DATASET =
+ new MetadataIndex(PROPERTIES_DATAVERSE, 2, new IAType[] { BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME)), 0,
+ MetadataRecordTypes.DATAVERSE_RECORDTYPE, true, new int[] { 0 });
+ public static final IMetadataIndex DATASET_DATASET =
+ new MetadataIndex(PROPERTIES_DATASET, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATASET_NAME)),
+ 0, MetadataRecordTypes.DATASET_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex DATATYPE_DATASET =
+ new MetadataIndex(PROPERTIES_DATATYPE, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATATYPE_NAME)),
+ 0, MetadataRecordTypes.DATATYPE_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex INDEX_DATASET = new MetadataIndex(PROPERTIES_INDEX, 4,
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATASET_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_INDEX_NAME)),
+ 0, MetadataRecordTypes.INDEX_RECORDTYPE, true, new int[] { 0, 1, 2 });
+ public static final IMetadataIndex NODE_DATASET =
+ new MetadataIndex(PROPERTIES_NODE, 2, new IAType[] { BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_NODE_NAME)), 0,
+ MetadataRecordTypes.NODE_RECORDTYPE, true, new int[] { 0 });
+ public static final IMetadataIndex NODEGROUP_DATASET =
+ new MetadataIndex(PROPERTIES_NODEGROUP, 2, new IAType[] { BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_GROUP_NAME)), 0,
+ MetadataRecordTypes.NODEGROUP_RECORDTYPE, true, new int[] { 0 });
+ public static final IMetadataIndex FUNCTION_DATASET = new MetadataIndex(PROPERTIES_FUNCTION, 4,
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_ARITY)),
+ 0, MetadataRecordTypes.FUNCTION_RECORDTYPE, true, new int[] { 0, 1, 2 });
+ public static final IMetadataIndex DATASOURCE_ADAPTER_DATASET = new MetadataIndex(PROPERTIES_DATASOURCE_ADAPTER, 3,
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_NAME)),
+ 0, MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex LIBRARY_DATASET =
+ new MetadataIndex(PROPERTIES_LIBRARY, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_NAME)),
+ 0, MetadataRecordTypes.LIBRARY_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex FEED_DATASET =
+ new MetadataIndex(PROPERTIES_FEED, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_FEED_NAME)),
+ 0, MetadataRecordTypes.FEED_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex FEED_ACTIVITY_DATASET = null;
+ public static final IMetadataIndex FEED_POLICY_DATASET =
+ new MetadataIndex(PROPERTIES_FEED_POLICY, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_POLICY_NAME)),
+ 0, MetadataRecordTypes.FEED_POLICY_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex COMPACTION_POLICY_DATASET = new MetadataIndex(PROPERTIES_COMPACTION_POLICY, 3,
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_COMPACTION_POLICY)),
+ 0, MetadataRecordTypes.COMPACTION_POLICY_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex EXTERNAL_FILE_DATASET = new MetadataIndex(PROPERTIES_EXTERNAL_FILE, 4,
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATASET_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_FILE_NUMBER)),
+ 0, MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE, true, new int[] { 0, 1, 2 });
- /**
- * Create all metadata primary index descriptors. MetadataRecordTypes must
- * have been initialized before calling this init.
- * @throws MetadataException
- * If MetadataRecordTypes have not been initialized.
- */
- public static void init() throws MetadataException {
- // Make sure the MetadataRecordTypes have been initialized.
- if (MetadataRecordTypes.DATASET_RECORDTYPE == null) {
- throw new MetadataException(
- "Must initialize MetadataRecordTypes before initializing MetadataPrimaryIndexes");
- }
-
- DATAVERSE_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.DATAVERSE, 2,
- new IAType[] { BuiltinType.ASTRING }, (Arrays.asList(Arrays.asList("DataverseName"))), 0,
- MetadataRecordTypes.DATAVERSE_RECORDTYPE, true, new int[] { 0 });
-
- DATASET_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.DATASET, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("DatasetName"))), 0,
- MetadataRecordTypes.DATASET_RECORDTYPE, true, new int[] { 0, 1 });
-
- DATATYPE_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.DATATYPE, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("DatatypeName"))), 0,
- MetadataRecordTypes.DATATYPE_RECORDTYPE, true, new int[] { 0, 1 });
-
- INDEX_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.INDEX, 4,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("DatasetName"),
- Arrays.asList("IndexName"))),
- 0, MetadataRecordTypes.INDEX_RECORDTYPE, true, new int[] { 0, 1, 2 });
-
- NODE_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.NODE, 2, new IAType[] { BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("NodeName"))), 0, MetadataRecordTypes.NODE_RECORDTYPE, true,
- new int[] { 0 });
-
- NODEGROUP_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.NODEGROUP, 2,
- new IAType[] { BuiltinType.ASTRING }, (Arrays.asList(Arrays.asList("GroupName"))), 0,
- MetadataRecordTypes.NODEGROUP_RECORDTYPE, true, new int[] { 0 });
-
- FUNCTION_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.FUNCTION, 4,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("Name"), Arrays.asList("Arity"))), 0,
- MetadataRecordTypes.FUNCTION_RECORDTYPE, true, new int[] { 0, 1, 2 });
-
- DATASOURCE_ADAPTER_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.DATASOURCE_ADAPTER, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("Name"))), 0,
- MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, true,
- new int[] { 0, 1 });
-
- FEED_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.FEED, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("FeedName"))), 0,
- MetadataRecordTypes.FEED_RECORDTYPE, true, new int[] { 0, 1 });
-
- LIBRARY_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.LIBRARY, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("Name"))), 0,
- MetadataRecordTypes.LIBRARY_RECORDTYPE, true, new int[] { 0, 1 });
-
- FEED_POLICY_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.FEED_POLICY, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("PolicyName"))), 0,
- MetadataRecordTypes.FEED_POLICY_RECORDTYPE, true, new int[] { 0, 1 });
-
- COMPACTION_POLICY_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.COMPACTION_POLICY, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("CompactionPolicy"))), 0,
- MetadataRecordTypes.COMPACTION_POLICY_RECORDTYPE, true,
- new int[] { 0, 1 });
-
- EXTERNAL_FILE_DATASET = new MetadataIndex(MetadataIndexImmutableProperties.EXTERNAL_FILE, 4,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 },
- (Arrays.asList(Arrays.asList("DataverseName"), Arrays.asList("DatasetName"),
- Arrays.asList("FileNumber"))),
- 0, MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE, true, new int[] { 0, 1, 2 });
+ private MetadataPrimaryIndexes() {
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 14ef049..a783c63 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -510,7 +510,7 @@
private MetadataRecordTypes() {
}
- private static ARecordType createRecordType(String recordTypeName, String[] fieldNames, IAType[] fieldTypes,
+ public static ARecordType createRecordType(String recordTypeName, String[] fieldNames, IAType[] fieldTypes,
boolean isOpen) {
ARecordType recordType = new ARecordType(recordTypeName, fieldNames, fieldTypes, isOpen);
if (recordTypeName != null) {
@@ -519,7 +519,7 @@
return recordType;
}
- private static final ARecordType createPropertiesRecordType() {
+ public static final ARecordType createPropertiesRecordType() {
return createRecordType(
// RecordTypeName
null,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java
index d809f4f..1272d03 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java
@@ -27,11 +27,15 @@
import java.util.Set;
import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
@@ -43,6 +47,9 @@
import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
public abstract class AqlDataSource implements IDataSource<AqlSourceId> {
@@ -217,4 +224,10 @@
return new ArrayList<>(dataScanVariables.subList(0, dataScanVariables.size() - (hasMeta() ? 2 : 1)));
}
+ public abstract Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ AqlMetadataProvider aqlMetadataProvider, IDataSource<AqlSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
+ throws AlgebricksException;
}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 679379b..da54e32 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -30,7 +30,6 @@
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
import org.apache.asterix.common.context.ITransactionSubsystemProvider;
import org.apache.asterix.common.context.TransactionSubsystemProvider;
@@ -50,7 +49,6 @@
import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.api.IDataSourceAdapter;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -58,7 +56,6 @@
import org.apache.asterix.external.operators.ExternalDataScanOperatorDescriptor;
import org.apache.asterix.external.operators.ExternalLookupOperatorDescriptor;
import org.apache.asterix.external.operators.ExternalRTreeSearchOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
import org.apache.asterix.external.provider.AdapterFactoryProvider;
import org.apache.asterix.external.util.ExternalDataConstants;
@@ -66,7 +63,6 @@
import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.AqlLinearizeComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.formats.nontagged.AqlTypeTraitProvider;
import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.MetadataManager;
@@ -83,10 +79,10 @@
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.entities.NodeGroup;
-import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
import org.apache.asterix.metadata.utils.DatasetUtils;
import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
@@ -97,7 +93,6 @@
import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.runtime.base.AsterixTupleFilterFactory;
import org.apache.asterix.runtime.formats.FormatUtils;
-import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.asterix.runtime.operators.AsterixLSMInvertedIndexUpsertOperatorDescriptor;
import org.apache.asterix.runtime.operators.AsterixLSMTreeUpsertOperatorDescriptor;
@@ -334,110 +329,31 @@
List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
try {
- switch (((AqlDataSource) dataSource).getDatasourceType()) {
- case FEED:
- return buildFeedCollectRuntime(jobSpec, (FeedDataSource) dataSource);
- case INTERNAL_DATASET: {
- // querying an internal dataset
- return buildInternalDatasetScan(jobSpec, scanVariables, minFilterVars, maxFilterVars, opSchema,
- typeEnv, dataSource, context, implConfig);
- }
- case EXTERNAL_DATASET: {
- // querying an external dataset
- Dataset dataset = ((DatasetDataSource) dataSource).getDataset();
- String itemTypeName = dataset.getItemTypeName();
- IAType itemType = MetadataManager.INSTANCE
- .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
-
- ExternalDatasetDetails edd = (ExternalDatasetDetails) dataset.getDatasetDetails();
- IAdapterFactory adapterFactory = getConfiguredAdapterFactory(dataset, edd.getAdapter(),
- edd.getProperties(), (ARecordType) itemType, false, null, null);
- return buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
- NonTaggedDataFormat.INSTANCE);
- }
- case LOADABLE: {
- // This is a load into dataset operation
- LoadableDataSource alds = (LoadableDataSource) dataSource;
- List<List<String>> partitioningKeys = alds.getPartitioningKeys();
- boolean isPKAutoGenerated =
- ((InternalDatasetDetails) alds.getTargetDataset().getDatasetDetails()).isAutogenerated();
- ARecordType itemType = (ARecordType) alds.getLoadedType();
- int pkIndex = 0;
- IAdapterFactory adapterFactory =
- getConfiguredAdapterFactory(alds.getTargetDataset(), alds.getAdapter(),
- alds.getAdapterProperties(), itemType, isPKAutoGenerated, partitioningKeys, null);
- RecordDescriptor rDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
- return buildLoadableDatasetScan(jobSpec, alds, adapterFactory, rDesc, isPKAutoGenerated,
- partitioningKeys, itemType, pkIndex);
- }
- default: {
- throw new IllegalArgumentException();
- }
-
- }
+ return ((AqlDataSource) dataSource).buildDatasourceScanRuntime(this, dataSource, scanVariables,
+ projectVariables, projectPushed, minFilterVars, maxFilterVars, opSchema, typeEnv, context, jobSpec,
+ implConfig);
} catch (AsterixException e) {
throw new AlgebricksException(e);
}
}
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedCollectRuntime(JobSpecification jobSpec,
- FeedDataSource feedDataSource) throws AlgebricksException {
-
- try {
- ARecordType feedOutputType = (ARecordType) feedDataSource.getItemType();
- ISerializerDeserializer payloadSerde =
- NonTaggedDataFormat.INSTANCE.getSerdeProvider().getSerializerDeserializer(feedOutputType);
- IAType metaType = feedDataSource.getMetaItemType();
- List<IAType> pkTypes = feedDataSource.getPkTypes();
- ArrayList<ISerializerDeserializer> serdes = new ArrayList<>();
- serdes.add(payloadSerde);
- if (metaType != null) {
- serdes.add(AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType));
- }
- if (pkTypes != null) {
- for (IAType type : pkTypes) {
- serdes.add(AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(type));
- }
- }
- RecordDescriptor feedDesc =
- new RecordDescriptor(serdes.toArray(new ISerializerDeserializer[serdes.size()]));
- FeedPolicyEntity feedPolicy =
- (FeedPolicyEntity) feedDataSource.getProperties().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
- if (feedPolicy == null) {
- throw new AlgebricksException("Feed not configured with a policy");
- }
- feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
- FeedConnectionId feedConnectionId = new FeedConnectionId(feedDataSource.getId().getDataverseName(),
- feedDataSource.getId().getDatasourceName(), feedDataSource.getTargetDataset());
- FeedCollectOperatorDescriptor feedCollector =
- new FeedCollectOperatorDescriptor(jobSpec, feedConnectionId, feedDataSource.getSourceFeedId(),
- feedOutputType, feedDesc, feedPolicy.getProperties(), feedDataSource.getLocation());
-
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedCollector,
- determineLocationConstraint(feedDataSource));
-
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
- }
-
- private AlgebricksAbsolutePartitionConstraint determineLocationConstraint(FeedDataSource feedDataSource)
+ public static AlgebricksAbsolutePartitionConstraint determineLocationConstraint(FeedDataSource feedDataSource)
throws AsterixException {
return new AlgebricksAbsolutePartitionConstraint(feedDataSource.getLocations());
}
- private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(JobSpecification jobSpec,
- LoadableDataSource alds, IAdapterFactory adapterFactory, RecordDescriptor rDesc, boolean isPKAutoGenerated,
- List<List<String>> primaryKeys, ARecordType recType, int pkIndex) throws AlgebricksException {
- ExternalDataScanOperatorDescriptor dataScanner =
- new ExternalDataScanOperatorDescriptor(jobSpec, rDesc, adapterFactory);
+ protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
+ JobSpecification jobSpec, IAdapterFactory adapterFactory, RecordDescriptor rDesc)
+ throws AlgebricksException {
+ ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, rDesc,
+ adapterFactory);
AlgebricksPartitionConstraint constraint;
try {
constraint = adapterFactory.getPartitionConstraint();
} catch (Exception e) {
throw new AlgebricksException(e);
}
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
+ return new Pair<>(dataScanner, constraint);
}
public IDataFormat getDataFormat(String dataverseName) throws AsterixException {
@@ -451,40 +367,7 @@
return format;
}
- private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
- List<LogicalVariable> outputVars, List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
- IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, IDataSource<AqlSourceId> dataSource,
- JobGenContext context, Object implConfig) throws AlgebricksException, MetadataException {
- AqlSourceId asid = dataSource.getId();
- String dataverseName = asid.getDataverseName();
- String datasetName = asid.getDatasourceName();
- Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
-
- int[] minFilterFieldIndexes = null;
- if (minFilterVars != null && !minFilterVars.isEmpty()) {
- minFilterFieldIndexes = new int[minFilterVars.size()];
- int i = 0;
- for (LogicalVariable v : minFilterVars) {
- minFilterFieldIndexes[i] = opSchema.findVariable(v);
- i++;
- }
- }
- int[] maxFilterFieldIndexes = null;
- if (maxFilterVars != null && !maxFilterVars.isEmpty()) {
- maxFilterFieldIndexes = new int[maxFilterVars.size()];
- int i = 0;
- for (LogicalVariable v : maxFilterVars) {
- maxFilterFieldIndexes[i] = opSchema.findVariable(v);
- i++;
- }
- }
-
- return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, true, false,
- ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true, true,
- implConfig, minFilterFieldIndexes, maxFilterFieldIndexes);
- }
-
- private IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
+ protected IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
Map<String, String> configuration, ARecordType itemType, boolean isPKAutoGenerated,
List<List<String>> primaryKeys, ARecordType metaType) throws AlgebricksException {
try {
@@ -493,9 +376,9 @@
configuration, itemType, metaType);
// check to see if dataset is indexed
- Index filesIndex =
- MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName(),
- dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
+ Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(),
+ dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
if (filesIndex != null && filesIndex.getPendingOp() == 0) {
// get files
@@ -526,8 +409,8 @@
ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
- ExternalDataScanOperatorDescriptor dataScanner =
- new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc, adapterFactory);
+ ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc,
+ adapterFactory);
AlgebricksPartitionConstraint constraint;
try {
@@ -542,8 +425,8 @@
public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> buildFeedIntakeRuntime(
JobSpecification jobSpec, Feed primaryFeed, FeedPolicyAccessor policyAccessor) throws Exception {
Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> factoryOutput = null;
- factoryOutput =
- FeedMetadataUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx, libraryManager);
+ factoryOutput = FeedMetadataUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx,
+ libraryManager);
ARecordType recordType = FeedMetadataUtil.getOutputType(primaryFeed, primaryFeed.getAdapterConfiguration(),
ExternalDataConstants.KEY_TYPE_NAME);
IAdapterFactory adapterFactory = factoryOutput.first;
@@ -587,18 +470,19 @@
ITypeTraits[] typeTraits;
IBinaryComparatorFactory[] comparatorFactories;
- ARecordType itemType =
- (ARecordType) this.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+ ARecordType itemType = (ARecordType) this.findType(dataset.getItemTypeDataverseName(),
+ dataset.getItemTypeName());
ARecordType metaType = null;
List<Integer> primaryKeyIndicators = null;
if (dataset.hasMetaPart()) {
- metaType =
- (ARecordType) findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
+ metaType = (ARecordType) findType(dataset.getMetaItemTypeDataverseName(),
+ dataset.getMetaItemTypeName());
primaryKeyIndicators = ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
}
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
int[] btreeFields = null;
@@ -612,11 +496,11 @@
bloomFilterKeyFields[i] = i;
}
Pair<IBinaryComparatorFactory[], ITypeTraits[]> comparatorFactoriesAndTypeTraits =
- getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(secondaryIndex.getIndexType(),
- secondaryIndex.getKeyFieldNames(), secondaryIndex.getKeyFieldTypes(),
- DatasetUtils.getPartitioningKeys(dataset), itemType, dataset.getDatasetType(),
- dataset.hasMetaPart(), primaryKeyIndicators,
- secondaryIndex.getKeyFieldSourceIndicators(), metaType);
+ getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(
+ secondaryIndex.getIndexType(), secondaryIndex.getKeyFieldNames(),
+ secondaryIndex.getKeyFieldTypes(), DatasetUtils.getPartitioningKeys(dataset), itemType,
+ dataset.getDatasetType(), dataset.hasMetaPart(), primaryKeyIndicators,
+ secondaryIndex.getKeyFieldSourceIndicators(), metaType);
comparatorFactories = comparatorFactoriesAndTypeTraits.first;
typeTraits = comparatorFactoriesAndTypeTraits.second;
if (filterTypeTraits != null) {
@@ -636,8 +520,8 @@
// get meta item type
ARecordType metaItemType = DatasetUtils.getMetaType(this, dataset);
typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, metaItemType);
- comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset, itemType, metaItemType,
- context.getBinaryComparatorFactoryProvider());
+ comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset, itemType,
+ metaItemType, context.getBinaryComparatorFactoryProvider());
filterFields = DatasetUtils.createFilterFields(dataset);
btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
}
@@ -673,8 +557,8 @@
: new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
txnSubsystemProvider, ResourceType.LSM_BTREE);
}
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
BTreeSearchOperatorDescriptor btreeSearchOp;
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
@@ -696,12 +580,13 @@
// Be Careful of Key Start Index ?
int[] buddyBreeFields = new int[] { numSecondaryKeys };
ExternalBTreeWithBuddyDataflowHelperFactory indexDataflowHelperFactory =
- new ExternalBTreeWithBuddyDataflowHelperFactory(compactionInfo.first, compactionInfo.second,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
- getStorageProperties().getBloomFilterFalsePositiveRate(), buddyBreeFields,
- ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this), !temp);
+ new ExternalBTreeWithBuddyDataflowHelperFactory(
+ compactionInfo.first, compactionInfo.second,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
+ getStorageProperties().getBloomFilterFalsePositiveRate(), buddyBreeFields,
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this), !temp);
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, rtcProvider,
rtcProvider, spPc.first, typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexDataflowHelperFactory, retainInput,
@@ -730,12 +615,12 @@
int i = 0;
for (; i < sidxKeyFieldCount; ++i) {
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i), sidxKeyFieldNames.get(i),
- (hasMeta && secondaryIndexIndicators.get(i).intValue() == 1) ? metaType : recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i),
+ sidxKeyFieldNames.get(i),
+ (hasMeta && secondaryIndexIndicators.get(i).intValue() == 1) ? metaType : recType);
IAType keyType = keyPairType.first;
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
+ true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
@@ -757,8 +642,8 @@
} catch (AsterixException e) {
throw new AlgebricksException(e);
}
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
+ true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
@@ -771,7 +656,8 @@
int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
try {
- ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+ ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(),
+ dataset.getItemTypeName());
int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
boolean temp = dataset.getDatasetDetails().isTemp();
@@ -786,10 +672,11 @@
int numSecondaryKeys = secondaryKeyFields.size();
if (numSecondaryKeys != 1) {
throw new AlgebricksException("Cannot use " + numSecondaryKeys
- + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
+ + " fields as a key for the R-tree index. "
+ + "There can be only one field as a key for the R-tree index.");
}
- Pair<IAType, Boolean> keyTypePair =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+ secondaryKeyFields.get(0), recType);
IAType keyType = keyTypePair.first;
if (keyType == null) {
throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
@@ -815,12 +702,12 @@
numNestedSecondaryKeyFields + numPrimaryKeys, typeEnv, context);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
- splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(), dataset.getDatasetName(),
- indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataset.getDataverseName(), dataset.getDatasetName(), indexName, temp);
ARecordType metaType = null;
if (dataset.hasMetaPart()) {
- metaType =
- (ARecordType) findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
+ metaType = (ARecordType) findType(dataset.getMetaItemTypeDataverseName(),
+ dataset.getMetaItemTypeName());
}
IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
@@ -831,7 +718,8 @@
}
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
recType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
int[] rtreeFields = null;
@@ -845,15 +733,15 @@
}
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
- ISearchOperationCallbackFactory searchCallbackFactory =
- temp ? NoOpOperationCallbackFactory.INSTANCE : new SecondaryIndexSearchOperationCallbackFactory();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
+ ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
+ : new SecondaryIndexSearchOperationCallbackFactory();
RTreeSearchOperatorDescriptor rtreeSearchOp;
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
- IBinaryComparatorFactory[] deletedKeyBTreeCompFactories =
- getMergedComparatorFactories(comparatorFactories, primaryComparatorFactories);
+ IBinaryComparatorFactory[] deletedKeyBTreeCompFactories = getMergedComparatorFactories(
+ comparatorFactories, primaryComparatorFactories);
IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
valueProviderFactories, RTreePolicyType.RTREE, deletedKeyBTreeCompFactories,
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
@@ -915,8 +803,8 @@
File outFile = fs.getLocalFile().getFile();
String nodeId = fs.getNodeName();
- SinkWriterRuntimeFactory runtime =
- new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, getWriterFactory(), inputDesc);
+ SinkWriterRuntimeFactory runtime = new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile,
+ getWriterFactory(), inputDesc);
AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeId });
return new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(runtime, apc);
}
@@ -1045,8 +933,8 @@
ARecordType metaType = null;
if (dataset.hasMetaPart()) {
- metaType =
- (ARecordType) findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
+ metaType = (ARecordType) findType(dataset.getMetaItemTypeDataverseName(),
+ dataset.getMetaItemTypeName());
}
String itemTypeName = dataset.getItemTypeName();
@@ -1057,14 +945,15 @@
itemType, metaType, context.getBinaryComparatorFactoryProvider());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
- indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataSource.getId().getDataverseName(), datasetName, indexName, temp);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
long numElementsHint = getCardinalityPerPartitionHint(dataset);
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = DatasetUtils.createFilterFields(dataset);
int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
@@ -1074,8 +963,8 @@
// right callback
// (ex. what's the expected behavior when there is an error during
// bulkload?)
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec, null,
appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
@@ -1147,8 +1036,8 @@
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, metaItemType, context.getBinaryComparatorFactoryProvider());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
- indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataSource.getId().getDataverseName(), datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -1171,8 +1060,8 @@
: new PrimaryIndexModificationOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE, dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -1243,12 +1132,12 @@
AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
switch (secondaryIndex.getIndexType()) {
case BTREE: {
- return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv, primaryKeys,
+ return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec, indexOp,
bulkload);
}
case RTREE: {
- return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv, primaryKeys,
+ return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec, indexOp,
bulkload);
}
@@ -1469,8 +1358,8 @@
secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName,
- dataset.getDatasetDetails().isTemp());
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, dataset.getDatasetDetails().isTemp());
// Generate Output Record format
ISerializerDeserializer<?>[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
@@ -1539,13 +1428,13 @@
return null;
}
IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
- IScalarEvaluatorFactory filterEvalFactory =
- expressionRuntimeProvider.createEvaluatorFactory(filterExpr, typeEnv, inputSchemas, context);
+ IScalarEvaluatorFactory filterEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(filterExpr,
+ typeEnv, inputSchemas, context);
return new AsterixTupleFilterFactory(filterEvalFactory, context.getBinaryBooleanInspectorFactory());
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String dataverseName,
- String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ String datasetName, String indexName, IOperatorSchema propagatedSchema,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
@@ -1602,7 +1491,8 @@
dataset.getDatasetName(), indexName);
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
recType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
int[] btreeFields = null;
@@ -1620,25 +1510,26 @@
ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
for (i = 0; i < secondaryKeys.size(); ++i) {
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i), secondaryKeyNames.get(i), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i),
+ secondaryKeyNames.get(i), recType);
IAType keyType = keyPairType.first;
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
+ true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
for (List<String> partitioningKey : partitioningKeys) {
IAType keyType = recType.getSubFieldType(partitioningKey);
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
+ true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
++i;
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -1651,8 +1542,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE,
dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -1792,8 +1683,8 @@
IAType secondaryKeyType = null;
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+ secondaryKeyExprs.get(0), recType);
secondaryKeyType = keyPairType.first;
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
@@ -1842,7 +1733,8 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -1856,8 +1748,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
ResourceType.LSM_INVERTED_INDEX, dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory indexDataFlowFactory;
if (!isPartitioned) {
indexDataFlowFactory = new LSMInvertedIndexDataflowHelperFactory(
@@ -1899,7 +1791,7 @@
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeDmlRuntime(String dataverseName,
- String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ String datasetName, String indexName, IOperatorSchema propagatedSchema,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
@@ -1921,11 +1813,11 @@
dataset.getDatasetName(), indexName);
List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+ secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
- boolean isPointMBR =
- spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
+ boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT
+ || spatialType.getTypeTag() == ATypeTag.POINT3D;
int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numSecondaryKeys = dimension * 2;
int numPrimaryKeys = primaryKeys.size();
@@ -1960,8 +1852,8 @@
IPrimitiveValueProviderFactory[] valueProviderFactories =
new IPrimitiveValueProviderFactory[numSecondaryKeys];
for (i = 0; i < numSecondaryKeys; i++) {
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(nestedKeyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(nestedKeyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
@@ -1976,14 +1868,16 @@
dataset, recType, metaItemType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, temp);
int[] btreeFields = new int[primaryComparatorFactories.length];
for (int k = 0; k < btreeFields.length; k++) {
btreeFields[k] = k + numSecondaryKeys;
}
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
recType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
int[] rtreeFields = null;
@@ -2002,16 +1896,18 @@
TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
IModificationOperationCallbackFactory modificationCallbackFactory = temp
? new TempDatasetSecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
- modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_RTREE)
+ modificationCallbackPrimaryKeyFields, txnSubsystemProvider,
+ indexOp, ResourceType.LSM_RTREE)
: new SecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
- modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_RTREE,
+ modificationCallbackPrimaryKeyFields, txnSubsystemProvider,
+ indexOp, ResourceType.LSM_RTREE,
dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
- IBinaryComparatorFactory[] deletedKeyBTreeCompFactories =
- getMergedComparatorFactories(comparatorFactories, primaryComparatorFactories);
+ IBinaryComparatorFactory[] deletedKeyBTreeCompFactories = getMergedComparatorFactories(comparatorFactories,
+ primaryComparatorFactories);
IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
valueProviderFactories, RTreePolicyType.RTREE, deletedKeyBTreeCompFactories,
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
@@ -2069,8 +1965,8 @@
numElementsHint = Long.parseLong(numElementsHintString);
}
int numPartitions = 0;
- List<String> nodeGroup =
- MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName()).getNodeNames();
+ List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName())
+ .getNodeNames();
for (String nd : nodeGroup) {
numPartitions += AsterixClusterProperties.INSTANCE.getNodePartitionsCount(nd);
}
@@ -2112,8 +2008,8 @@
}
public Dataset findDataset(String dataverse, String dataset) throws AlgebricksException {
- String dv =
- dataverse == null ? (defaultDataverse == null ? null : defaultDataverse.getDataverseName()) : dataverse;
+ String dv = dataverse == null ? (defaultDataverse == null ? null : defaultDataverse.getDataverseName())
+ : dataverse;
if (dv == null) {
return null;
}
@@ -2244,8 +2140,8 @@
spPc = metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
dataset.getDatasetName(),
dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX), false);
- ISearchOperationCallbackFactory searchOpCallbackFactory =
- temp ? NoOpOperationCallbackFactory.INSTANCE : new SecondaryIndexSearchOperationCallbackFactory();
+ ISearchOperationCallbackFactory searchOpCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
+ : new SecondaryIndexSearchOperationCallbackFactory();
// Create the operator
ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
outRecDesc, indexDataflowHelperFactory, retainInput, appContext.getIndexLifecycleManagerProvider(),
@@ -2319,8 +2215,8 @@
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, metaItemType, context.getBinaryComparatorFactoryProvider());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
- indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataSource.getId().getDataverseName(), datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -2331,7 +2227,8 @@
}
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = DatasetUtils.createFilterFields(dataset);
int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
@@ -2346,8 +2243,8 @@
LockThenSearchOperationCallbackFactory searchCallbackFactory = new LockThenSearchOperationCallbackFactory(
jobId, datasetId, primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -2356,24 +2253,24 @@
btreeFields, filterFields, !temp);
AsterixLSMTreeUpsertOperatorDescriptor op;
- ITypeTraits[] outputTypeTraits =
- new ITypeTraits[recordDesc.getFieldCount() + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+ ITypeTraits[] outputTypeTraits = new ITypeTraits[recordDesc.getFieldCount()
+ + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
ISerializerDeserializer[] outputSerDes = new ISerializerDeserializer[recordDesc.getFieldCount()
+ (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
for (int j = 0; j < recordDesc.getFieldCount(); j++) {
outputTypeTraits[j] = recordDesc.getTypeTraits()[j];
outputSerDes[j] = recordDesc.getFields()[j];
}
- outputSerDes[outputSerDes.length - (dataset.hasMetaPart() ? 2 : 1) - numFilterFields] =
- FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(itemType);
- outputTypeTraits[outputTypeTraits.length - (dataset.hasMetaPart() ? 2 : 1) - numFilterFields] =
- FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(itemType);
+ outputSerDes[outputSerDes.length - (dataset.hasMetaPart() ? 2 : 1) - numFilterFields] = FormatUtils
+ .getDefaultFormat().getSerdeProvider().getSerializerDeserializer(itemType);
+ outputTypeTraits[outputTypeTraits.length - (dataset.hasMetaPart() ? 2 : 1) - numFilterFields] = FormatUtils
+ .getDefaultFormat().getTypeTraitProvider().getTypeTrait(itemType);
if (dataset.hasMetaPart()) {
- outputSerDes[outputSerDes.length - 1 - numFilterFields] =
- FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
- outputTypeTraits[outputTypeTraits.length - 1 - numFilterFields] =
- FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
+ outputSerDes[outputSerDes.length - 1 - numFilterFields] = FormatUtils.getDefaultFormat()
+ .getSerdeProvider().getSerializerDeserializer(metaItemType);
+ outputTypeTraits[outputTypeTraits.length - 1 - numFilterFields] = FormatUtils.getDefaultFormat()
+ .getTypeTraitProvider().getTypeTrait(metaItemType);
}
int fieldIdx = -1;
@@ -2399,7 +2296,7 @@
searchCallbackFactory, null);
op.setType(itemType);
op.setFilterIndex(fieldIdx);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, splitsAndConstraint.second);
+ return new Pair<>(op, splitsAndConstraint.second);
} catch (MetadataException me) {
throw new AlgebricksException(me);
@@ -2433,7 +2330,7 @@
AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
ArrayList<LogicalVariable> prevAdditionalFilteringKeys = null;
if (prevAdditionalFilteringKey != null) {
- prevAdditionalFilteringKeys = new ArrayList<LogicalVariable>();
+ prevAdditionalFilteringKeys = new ArrayList<>();
prevAdditionalFilteringKeys.add(prevAdditionalFilteringKey);
}
switch (secondaryIndex.getIndexType()) {
@@ -2451,7 +2348,7 @@
case SINGLE_PARTITION_NGRAM_INVIX:
case LENGTH_PARTITIONED_WORD_INVIX:
case LENGTH_PARTITIONED_NGRAM_INVIX: {
- return getInvertedIndexUpsertRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
+ return getInvertedIndexUpsertRuntime(dataverseName, datasetName, indexName, propagatedSchema,
primaryKeys, secondaryKeys, additionalFilteringKeys, filterFactory, recordDesc, context, spec,
secondaryIndex.getIndexType(), prevSecondaryKeys, prevAdditionalFilteringKeys);
}
@@ -2464,7 +2361,7 @@
//TODO: refactor this method
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInvertedIndexUpsertRuntime(String dataverseName,
- String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ String datasetName, String indexName, IOperatorSchema propagatedSchema,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalFilteringKeys, AsterixTupleFilterFactory filterFactory,
RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexType indexType,
@@ -2590,8 +2487,8 @@
IAType secondaryKeyType = null;
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+ secondaryKeyExprs.get(0), recType);
secondaryKeyType = keyPairType.first;
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
@@ -2615,7 +2512,8 @@
secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
recType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
@@ -2640,7 +2538,8 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -2654,8 +2553,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, IndexOperation.UPSERT,
ResourceType.LSM_INVERTED_INDEX, dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory indexDataFlowFactory;
if (!isPartitioned) {
indexDataFlowFactory = new LSMInvertedIndexDataflowHelperFactory(
@@ -2713,12 +2612,12 @@
List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+ secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
- boolean isPointMBR =
- spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
+ boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT
+ || spatialType.getTypeTag() == ATypeTag.POINT3D;
int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numSecondaryKeys = dimension * 2;
int numPrimaryKeys = primaryKeys.size();
@@ -2775,8 +2674,8 @@
IPrimitiveValueProviderFactory[] valueProviderFactories =
new IPrimitiveValueProviderFactory[numSecondaryKeys];
for (i = 0; i < numSecondaryKeys; i++) {
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(nestedKeyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(nestedKeyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
@@ -2792,14 +2691,16 @@
dataset, recType, metaItemType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, temp);
int[] btreeFields = new int[primaryComparatorFactories.length];
for (int k = 0; k < btreeFields.length; k++) {
btreeFields[k] = k + numSecondaryKeys;
}
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
recType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
int[] rtreeFields = null;
@@ -2824,10 +2725,10 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, IndexOperation.UPSERT,
ResourceType.LSM_RTREE, dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
- IBinaryComparatorFactory[] deletedKeyBTreeCompFactories =
- getMergedComparatorFactories(comparatorFactories, primaryComparatorFactories);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
+ IBinaryComparatorFactory[] deletedKeyBTreeCompFactories = getMergedComparatorFactories(comparatorFactories,
+ primaryComparatorFactories);
IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
valueProviderFactories, RTreePolicyType.RTREE, deletedKeyBTreeCompFactories,
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
@@ -2926,7 +2827,8 @@
dataset.getDatasetName(), indexName);
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
- IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+ IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.
+ computeFilterBinaryComparatorFactories(dataset,
recType, context.getBinaryComparatorFactoryProvider());
int[] filterFields = null;
int[] btreeFields = null;
@@ -2944,25 +2846,26 @@
ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
for (i = 0; i < secondaryKeys.size(); ++i) {
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i), secondaryKeyNames.get(i), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i),
+ secondaryKeyNames.get(i), recType);
IAType keyType = keyPairType.first;
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.
+ getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
for (List<String> partitioningKey : partitioningKeys) {
IAType keyType = recType.getSubFieldType(partitioningKey);
- comparatorFactories[i] =
- AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.
+ getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
++i;
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+ splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -2976,8 +2879,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, IndexOperation.UPSERT,
ResourceType.LSM_BTREE, dataset.hasMetaPart());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
- DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
+ .getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -2989,7 +2892,7 @@
splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
idfh, filterFactory, false, indexName, null, modificationCallbackFactory,
NoOpOperationCallbackFactory.INSTANCE, prevFieldPermutation);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, splitsAndConstraint.second);
+ return new Pair<>(op, splitsAndConstraint.second);
} catch (Exception e) {
throw new AlgebricksException(e);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 4148d65..5a601bc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -20,14 +20,28 @@
import java.util.List;
+import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.metadata.IDatasetDetails;
+import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
+import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
public class DatasetDataSource extends AqlDataSource {
@@ -35,7 +49,7 @@
public DatasetDataSource(AqlSourceId id, Dataset dataset, IAType itemType, IAType metaItemType,
AqlDataSourceType datasourceType, IDatasetDetails datasetDetails, INodeDomain datasetDomain)
- throws AlgebricksException {
+ throws AlgebricksException {
super(id, itemType, metaItemType, datasourceType, datasetDomain);
this.dataset = dataset;
switch (dataset.getDatasetType()) {
@@ -57,8 +71,8 @@
InternalDatasetDetails internalDatasetDetails = (InternalDatasetDetails) datasetDetails;
ARecordType recordType = (ARecordType) itemType;
ARecordType metaRecordType = (ARecordType) metaItemType;
- List<IAType> partitioningKeyTypes = KeyFieldTypeUtils.getPartitioningKeyTypes(internalDatasetDetails,
- recordType, metaRecordType);
+ List<IAType> partitioningKeyTypes =
+ KeyFieldTypeUtils.getPartitioningKeyTypes(internalDatasetDetails, recordType, metaRecordType);
int n = partitioningKeyTypes.size();
schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
for (int keyIndex = 0; keyIndex < n; ++keyIndex) {
@@ -75,4 +89,56 @@
schemaTypes[0] = itemType;
}
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ AqlMetadataProvider aqlMetadataProvider, IDataSource<AqlSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
+ throws AlgebricksException {
+ switch (dataset.getDatasetType()) {
+ case EXTERNAL:
+ Dataset externalDataset = ((DatasetDataSource) dataSource).getDataset();
+ String itemTypeName = externalDataset.getItemTypeName();
+ IAType itemType = MetadataManager.INSTANCE.getDatatype(aqlMetadataProvider.getMetadataTxnContext(),
+ externalDataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
+
+ ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
+ IAdapterFactory adapterFactory = aqlMetadataProvider.getConfiguredAdapterFactory(externalDataset,
+ edd.getAdapter(), edd.getProperties(), (ARecordType) itemType, false, null, null);
+ return aqlMetadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
+ NonTaggedDataFormat.INSTANCE);
+ case INTERNAL:
+ AqlSourceId asid = getId();
+ String dataverseName = asid.getDataverseName();
+ String datasetName = asid.getDatasourceName();
+ Index primaryIndex = MetadataManager.INSTANCE.getIndex(aqlMetadataProvider.getMetadataTxnContext(),
+ dataverseName, datasetName, datasetName);
+
+ int[] minFilterFieldIndexes = null;
+ if (minFilterVars != null && !minFilterVars.isEmpty()) {
+ minFilterFieldIndexes = new int[minFilterVars.size()];
+ int i = 0;
+ for (LogicalVariable v : minFilterVars) {
+ minFilterFieldIndexes[i] = opSchema.findVariable(v);
+ i++;
+ }
+ }
+ int[] maxFilterFieldIndexes = null;
+ if (maxFilterVars != null && !maxFilterVars.isEmpty()) {
+ maxFilterFieldIndexes = new int[maxFilterVars.size()];
+ int i = 0;
+ for (LogicalVariable v : maxFilterVars) {
+ maxFilterFieldIndexes[i] = opSchema.findVariable(v);
+ i++;
+ }
+ }
+ return aqlMetadataProvider.buildBtreeRuntime(jobSpec, scanVariables, opSchema, typeEnv, context, true,
+ false, ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null,
+ true, true, implConfig, minFilterFieldIndexes, maxFilterFieldIndexes);
+ default:
+ throw new AlgebricksException("Unknown datasource type");
+ }
+ }
+
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 48f33ed..0a81f03 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -18,20 +18,39 @@
*/
package org.apache.asterix.metadata.declared;
+import java.util.ArrayList;
import java.util.List;
import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.api.IFeed;
+import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
+import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
-public class FeedDataSource extends AqlDataSource {
+public class FeedDataSource extends AqlDataSource implements IMutationDataSource {
private final Feed feed;
private final EntityId sourceFeedId;
@@ -113,6 +132,7 @@
return pkTypes;
}
+ @Override
public List<ScalarFunctionCallExpression> getKeyAccessExpression() {
return keyAccessExpression;
}
@@ -127,10 +147,12 @@
return dataScanVariables.get(0);
}
+ @Override
public boolean isChange() {
return pkTypes != null;
}
+ @Override
public List<LogicalVariable> getPkVars(List<LogicalVariable> allVars) {
if (pkTypes == null) {
return null;
@@ -141,4 +163,45 @@
return allVars.subList(1, allVars.size());
}
}
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ AqlMetadataProvider aqlMetadataProvider, IDataSource<AqlSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
+ throws AlgebricksException {
+ try {
+ ARecordType feedOutputType = (ARecordType) itemType;
+ ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
+ .getSerializerDeserializer(feedOutputType);
+ ArrayList<ISerializerDeserializer> serdes = new ArrayList<>();
+ serdes.add(payloadSerde);
+ if (metaItemType != null) {
+ serdes.add(AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaItemType));
+ }
+ if (pkTypes != null) {
+ for (IAType type : pkTypes) {
+ serdes.add(AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(type));
+ }
+ }
+ RecordDescriptor feedDesc = new RecordDescriptor(
+ serdes.toArray(new ISerializerDeserializer[serdes.size()]));
+ FeedPolicyEntity feedPolicy = (FeedPolicyEntity) getProperties()
+ .get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+ if (feedPolicy == null) {
+ throw new AlgebricksException("Feed not configured with a policy");
+ }
+ feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
+ FeedConnectionId feedConnectionId = new FeedConnectionId(getId().getDataverseName(),
+ getId().getDatasourceName(), getTargetDataset());
+ FeedCollectOperatorDescriptor feedCollector = new FeedCollectOperatorDescriptor(jobSpec, feedConnectionId,
+ getSourceFeedId(), feedOutputType, feedDesc, feedPolicy.getProperties(), getLocation());
+
+ return new Pair<>(feedCollector, new AlgebricksAbsolutePartitionConstraint(getLocations()));
+
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/IMutationDataSource.java
similarity index 64%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
copy to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/IMutationDataSource.java
index 1752054..2feb652 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/IMutationDataSource.java
@@ -16,12 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.metadata.declared;
-import java.util.Map;
+import java.util.List;
-public interface ISubscriberRuntime {
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
- public Map<String, String> getFeedPolicy();
+public interface IMutationDataSource {
+
+ boolean isChange();
+
+ List<LogicalVariable> getPkVars(List<LogicalVariable> allVars);
+
+ List<ScalarFunctionCallExpression> getKeyAccessExpression();
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index db4300a..2ffaded 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -24,6 +24,7 @@
import java.util.Map;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.utils.DatasetUtils;
@@ -31,8 +32,19 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.IAType;
import org.apache.commons.lang.StringUtils;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
public class LoadableDataSource extends AqlDataSource {
@@ -112,4 +124,24 @@
public Dataset getTargetDataset() {
return targetDataset;
}
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ AqlMetadataProvider aqlMetadataProvider, IDataSource<AqlSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
+ throws AlgebricksException {
+ // This is a load into dataset operation
+ LoadableDataSource alds = (LoadableDataSource) dataSource;
+ List<List<String>> aldsPartitioningKeys = alds.getPartitioningKeys();
+ boolean isAldsPKAutoGenerated = ((InternalDatasetDetails) alds.getTargetDataset().getDatasetDetails())
+ .isAutogenerated();
+ ARecordType itemType = (ARecordType) alds.getLoadedType();
+ IAdapterFactory adapterFactory = aqlMetadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
+ alds.getAdapter(), alds.getAdapterProperties(), itemType, isAldsPKAutoGenerated, aldsPartitioningKeys,
+ null);
+ RecordDescriptor rDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ return aqlMetadataProvider.buildLoadableDatasetScan(jobSpec, adapterFactory, rDesc);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index aba8f8d..baf141f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -25,14 +25,22 @@
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.MetadataCache;
import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
/**
* Metadata describing a dataset.
*/
public class Dataset implements IMetadataEntity<Dataset> {
- private static final long serialVersionUID = 1L;
+ /**
+ * Dataset related operations
+ */
+ public static final byte OP_READ = 0x00;
+ public static final byte OP_INSERT = 0x01;
+ public static final byte OP_DELETE = 0x02;
+ public static final byte OP_UPSERT = 0x03;
+ private static final long serialVersionUID = 1L;
private final String dataverseName;
// Enforced to be unique within a dataverse.
private final String datasetName;
@@ -84,6 +92,13 @@
this.hints = hints;
}
+ public Dataset(Dataset dataset) {
+ this(dataset.dataverseName, dataset.datasetName, dataset.itemTypeDataverseName, dataset.itemTypeName,
+ dataset.metaItemTypeDataverseName, dataset.metaItemTypeName, dataset.nodeGroupName,
+ dataset.compactionPolicy, dataset.compactionPolicyProperties, dataset.datasetDetails, dataset.hints,
+ dataset.datasetType, dataset.datasetId, dataset.pendingOp);
+ }
+
public String getDataverseName() {
return dataverseName;
}
@@ -176,6 +191,10 @@
return true;
}
+ public boolean allow(ProjectOperator project, byte operation) {
+ return !hasMetaPart();
+ }
+
@Override
public String toString() {
return dataverseName + "." + datasetName;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
index 467e6f5..1b40484 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
@@ -60,4 +60,9 @@
public Dataverse dropFromCache(MetadataCache cache) {
return cache.dropDataverse(this);
}
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + ":" + dataverseName;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
index 3508554..bbb62d3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
@@ -154,14 +154,14 @@
AMutableString aString = new AMutableString("");
AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
@SuppressWarnings("unchecked")
- ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ ISerializerDeserializer<ABoolean> booleanSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@SuppressWarnings("unchecked")
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AString> stringSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
@SuppressWarnings("unchecked")
- ISerializerDeserializer<AInt8> int8Serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
+ ISerializerDeserializer<AInt8> int8Serde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
// write field 0
fieldValue.reset();
@@ -280,8 +280,8 @@
propertyRecordBuilder.reset(recordType);
AMutableString aString = new AMutableString("");
@SuppressWarnings("unchecked")
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AString> stringSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
// write field 0
fieldValue.reset();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
index 62a28e9..5ec25e6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
@@ -38,6 +38,7 @@
* IMetadataEntityTupleTranslator.
*/
public abstract class AbstractTupleTranslator<T> implements IMetadataEntityTupleTranslator<T> {
+ private static final long serialVersionUID = 1L;
protected AMutableString aString = new AMutableString("");
@SuppressWarnings("unchecked")
protected ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -49,10 +50,10 @@
protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.AINT32);
- protected final IARecordBuilder recordBuilder;
- protected final ArrayBackedValueStorage fieldValue;
- protected final ArrayTupleBuilder tupleBuilder;
- protected final ArrayTupleReference tuple;
+ protected final transient IARecordBuilder recordBuilder;
+ protected final transient ArrayBackedValueStorage fieldValue;
+ protected final transient ArrayTupleBuilder tupleBuilder;
+ protected final transient ArrayTupleReference tuple;
public AbstractTupleTranslator(boolean getTuple, int fieldCount) {
if (getTuple) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
index e29b9dd..9d37ddb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
@@ -51,7 +51,7 @@
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.COMPACTION_POLICY_RECORDTYPE);
- public CompactionPolicyTupleTranslator(boolean getTuple) {
+ protected CompactionPolicyTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET.getFieldCount());
}
@@ -62,7 +62,7 @@
int recordLength = tuple.getFieldLength(COMPACTION_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
DataInput in = new DataInputStream(stream);
- ARecord compactionPolicyRecord = (ARecord) recordSerDes.deserialize(in);
+ ARecord compactionPolicyRecord = recordSerDes.deserialize(in);
return createCompactionPolicyFromARecord(compactionPolicyRecord);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 01380c0..e97f449 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -73,6 +73,7 @@
* Translates a Dataset metadata entity to an ITupleReference and vice versa.
*/
public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
+ private static final long serialVersionUID = 1L;
// Field indexes of serialized Dataset in a tuple.
// First key field.
public static final int DATASET_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
@@ -82,14 +83,14 @@
public static final int DATASET_PAYLOAD_TUPLE_FIELD_INDEX = 2;
@SuppressWarnings("unchecked")
- private final ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+ protected final ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.DATASET_RECORDTYPE);
- private final AMutableInt32 aInt32;
- protected final ISerializerDeserializer<AInt32> aInt32Serde;
- private final ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
+ protected final transient AMutableInt32 aInt32;
+ protected final transient ISerializerDeserializer<AInt32> aInt32Serde;
+ protected final transient ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
@SuppressWarnings("unchecked")
- public DatasetTupleTranslator(boolean getTuple) {
+ protected DatasetTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATASET_DATASET.getFieldCount());
aInt32 = new AMutableInt32(-1);
aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
@@ -106,7 +107,7 @@
return createDatasetFromARecord(datasetRecord);
}
- private Dataset createDatasetFromARecord(ARecord datasetRecord) throws IOException {
+ protected Dataset createDatasetFromARecord(ARecord datasetRecord) throws IOException {
String dataverseName = ((AString) datasetRecord
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
@@ -249,8 +250,7 @@
if (metaTypeDataverseNameIndex >= 0) {
metaTypeDataverseName = ((AString) datasetRecord.getValueByPos(metaTypeDataverseNameIndex))
.getStringValue();
- int metaTypeNameIndex = datasetRecord.getType()
- .getFieldIndex(MetadataRecordTypes.FIELD_NAME_METATYPE_NAME);
+ int metaTypeNameIndex = datasetRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_METATYPE_NAME);
metaTypeName = ((AString) datasetRecord.getValueByPos(metaTypeNameIndex)).getStringValue();
}
@@ -374,6 +374,18 @@
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_PENDINGOP_FIELD_INDEX, fieldValue);
// write open fields
+ writeOpenFields(dataset);
+
+ // write record
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+
+ protected void writeOpenFields(Dataset dataset) throws HyracksDataException {
+ // write open fields
if (dataset.hasMetaPart()) {
// write open field 1, the meta item type Dataverse name.
fieldName.reset();
@@ -393,16 +405,9 @@
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(fieldName, fieldValue);
}
-
- // write record
- recordBuilder.write(tupleBuilder.getDataOutput(), true);
- tupleBuilder.addFieldEndOffset();
-
- tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- return tuple;
}
- private void writeDatasetDetailsRecordType(IARecordBuilder recordBuilder, Dataset dataset, DataOutput dataOutput)
+ protected void writeDatasetDetailsRecordType(IARecordBuilder recordBuilder, Dataset dataset, DataOutput dataOutput)
throws HyracksDataException {
dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
@@ -417,7 +422,7 @@
}
- private Map<String, String> getDatasetHints(ARecord datasetRecord) {
+ protected Map<String, String> getDatasetHints(ARecord datasetRecord) {
Map<String, String> hints = new HashMap<String, String>();
String key;
String value;
@@ -434,7 +439,7 @@
}
@SuppressWarnings("unchecked")
- private void writeDatasetHintRecord(String name, String value, DataOutput out) throws HyracksDataException {
+ protected void writeDatasetHintRecord(String name, String value, DataOutput out) throws HyracksDataException {
IARecordBuilder propertyRecordBuilder = new RecordBuilder();
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
propertyRecordBuilder.reset(MetadataRecordTypes.DATASET_HINTS_RECORDTYPE);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index a824551..70c5813 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -52,7 +52,7 @@
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE);
- public DatasourceAdapterTupleTranslator(boolean getTuple) {
+ protected DatasourceAdapterTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET.getFieldCount());
}
@@ -63,7 +63,7 @@
int recordLength = tuple.getFieldLength(ADAPTER_PAYLOAD_TUPLE_FIELD_INDEX);
ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
DataInput in = new DataInputStream(stream);
- ARecord adapterRecord = (ARecord) recordSerDes.deserialize(in);
+ ARecord adapterRecord = recordSerDes.deserialize(in);
return createAdapterFromARecord(adapterRecord);
}
@@ -71,8 +71,9 @@
String dataverseName = ((AString) adapterRecord
.getValueByPos(MetadataRecordTypes.DATASOURCE_ADAPTER_ARECORD_DATAVERSENAME_FIELD_INDEX))
.getStringValue();
- String adapterName = ((AString) adapterRecord
- .getValueByPos(MetadataRecordTypes.DATASOURCE_ADAPTER_ARECORD_NAME_FIELD_INDEX)).getStringValue();
+ String adapterName =
+ ((AString) adapterRecord.getValueByPos(MetadataRecordTypes.DATASOURCE_ADAPTER_ARECORD_NAME_FIELD_INDEX))
+ .getStringValue();
String classname = ((AString) adapterRecord
.getValueByPos(MetadataRecordTypes.DATASOURCE_ADAPTER_ARECORD_CLASSNAME_FIELD_INDEX)).getStringValue();
IDataSourceAdapter.AdapterType adapterType = IDataSourceAdapter.AdapterType.valueOf(
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index 0604fff..b294b77 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -82,7 +82,7 @@
private final MetadataNode metadataNode;
private final JobId jobId;
- public DatatypeTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
+ protected DatatypeTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATATYPE_DATASET.getFieldCount());
this.jobId = jobId;
this.metadataNode = metadataNode;
@@ -100,15 +100,17 @@
}
private Datatype createDataTypeFromARecord(ARecord datatypeRecord) throws MetadataException {
- String dataverseName = ((AString) datatypeRecord
- .getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
- String datatypeName = ((AString) datatypeRecord
- .getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DATATYPENAME_FIELD_INDEX)).getStringValue();
+ String dataverseName =
+ ((AString) datatypeRecord.getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DATAVERSENAME_FIELD_INDEX))
+ .getStringValue();
+ String datatypeName =
+ ((AString) datatypeRecord.getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DATATYPENAME_FIELD_INDEX))
+ .getStringValue();
IAType type = AsterixBuiltinTypeMap.getBuiltinTypes().get(datatypeName);
if (type == null) {
// Derived Type
- ARecord derivedTypeRecord = (ARecord) datatypeRecord
- .getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DERIVED_FIELD_INDEX);
+ ARecord derivedTypeRecord =
+ (ARecord) datatypeRecord.getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DERIVED_FIELD_INDEX);
DerivedTypeTag tag = DerivedTypeTag.valueOf(
((AString) derivedTypeRecord.getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_TAG_FIELD_INDEX))
.getStringValue());
@@ -131,11 +133,11 @@
String fieldTypeName;
while (cursor.next()) {
ARecord field = (ARecord) cursor.get();
- fieldNames[fieldId] = ((AString) field
- .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDNAME_FIELD_INDEX))
+ fieldNames[fieldId] =
+ ((AString) field.getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDNAME_FIELD_INDEX))
.getStringValue();
- fieldTypeName = ((AString) field
- .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDTYPE_FIELD_INDEX))
+ fieldTypeName =
+ ((AString) field.getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDTYPE_FIELD_INDEX))
.getStringValue();
boolean isNullable = ((ABoolean) field
.getValueByPos(MetadataRecordTypes.FIELD_ARECORD_ISNULLABLE_FIELD_INDEX)).getBoolean()
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 74f85ab..086bb10 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -48,7 +48,7 @@
// Payload field containing serialized Dataverse.
public static final int DATAVERSE_PAYLOAD_TUPLE_FIELD_INDEX = 1;
- private AMutableInt32 aInt32;
+ private transient AMutableInt32 aInt32;
protected ISerializerDeserializer<AInt32> aInt32Serde;
@SuppressWarnings("unchecked")
@@ -56,7 +56,7 @@
.getSerializerDeserializer(MetadataRecordTypes.DATAVERSE_RECORDTYPE);
@SuppressWarnings("unchecked")
- public DataverseTupleTranslator(boolean getTuple) {
+ protected DataverseTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATAVERSE_DATASET.getFieldCount());
aInt32 = new AMutableInt32(-1);
aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
index 717e61f..f4100d4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
@@ -53,9 +53,9 @@
// Payload field containing serialized ExternalFile.
public static final int EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX = 3;
- protected AMutableInt32 aInt32 = new AMutableInt32(0);
- protected AMutableDateTime aDateTime = new AMutableDateTime(0);
- protected AMutableInt64 aInt64 = new AMutableInt64(0);
+ protected transient AMutableInt32 aInt32 = new AMutableInt32(0);
+ protected transient AMutableDateTime aDateTime = new AMutableDateTime(0);
+ protected transient AMutableInt64 aInt64 = new AMutableInt64(0);
@SuppressWarnings("unchecked")
protected ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -70,7 +70,7 @@
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE);
- public ExternalFileTupleTranslator(boolean getTuple) {
+ protected ExternalFileTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET.getFieldCount());
}
@@ -81,7 +81,7 @@
int recordLength = tuple.getFieldLength(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
DataInput in = new DataInputStream(stream);
- ARecord externalFileRecord = (ARecord) recordSerDes.deserialize(in);
+ ARecord externalFileRecord = recordSerDes.deserialize(in);
return createExternalFileFromARecord(externalFileRecord);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
index 4839065..5b9c1c9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
@@ -67,7 +67,7 @@
protected ISerializerDeserializer<AInt32> aInt32Serde;
@SuppressWarnings("unchecked")
- public FeedPolicyTupleTranslator(boolean getTuple) {
+ protected FeedPolicyTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.FEED_POLICY_DATASET.getFieldCount());
aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
}
@@ -173,8 +173,8 @@
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
propertyRecordBuilder.reset(MetadataRecordTypes.POLICY_PARAMS_RECORDTYPE);
AMutableString aString = new AMutableString("");
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AString> stringSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
// write field 0
fieldValue.reset();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
index fc82255..09b44b4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
@@ -67,10 +67,10 @@
public static final int FEED_PAYLOAD_TUPLE_FIELD_INDEX = 2;
@SuppressWarnings("unchecked")
- private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(MetadataRecordTypes.FEED_RECORDTYPE);
+ private ISerializerDeserializer<ARecord> recordSerDes =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.FEED_RECORDTYPE);
- public FeedTupleTranslator(boolean getTuple) {
+ protected FeedTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.FEED_DATASET.getFieldCount());
}
@@ -87,8 +87,9 @@
private Feed createFeedFromARecord(ARecord feedRecord) {
Feed feed = null;
- String dataverseName = ((AString) feedRecord
- .getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+ String dataverseName =
+ ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX))
+ .getStringValue();
String feedName = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX))
.getStringValue();
@@ -219,8 +220,8 @@
primaryDetailsRecordBuilder.reset(MetadataRecordTypes.PRIMARY_FEED_DETAILS_RECORDTYPE);
AMutableString aString = new AMutableString("");
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AString> stringSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
// write field 0
fieldValue.reset();
@@ -278,8 +279,8 @@
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
propertyRecordBuilder.reset(MetadataRecordTypes.FEED_ADAPTER_CONFIGURATION_RECORDTYPE);
AMutableString aString = new AMutableString("");
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AString> stringSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
// write field 0
fieldValue.reset();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index 72fe79d..f9e98e2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -60,7 +60,7 @@
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.FUNCTION_RECORDTYPE);
- public FunctionTupleTranslator(boolean getTuple) {
+ protected FunctionTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.FUNCTION_DATASET.getFieldCount());
}
@@ -71,15 +71,17 @@
int recordLength = frameTuple.getFieldLength(FUNCTION_PAYLOAD_TUPLE_FIELD_INDEX);
ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
DataInput in = new DataInputStream(stream);
- ARecord functionRecord = (ARecord) recordSerDes.deserialize(in);
+ ARecord functionRecord = recordSerDes.deserialize(in);
return createFunctionFromARecord(functionRecord);
}
private Function createFunctionFromARecord(ARecord functionRecord) {
- String dataverseName = ((AString) functionRecord
- .getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
- String functionName = ((AString) functionRecord
- .getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTIONNAME_FIELD_INDEX)).getStringValue();
+ String dataverseName =
+ ((AString) functionRecord.getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_DATAVERSENAME_FIELD_INDEX))
+ .getStringValue();
+ String functionName =
+ ((AString) functionRecord.getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTIONNAME_FIELD_INDEX))
+ .getStringValue();
String arity = ((AString) functionRecord
.getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_ARITY_FIELD_INDEX)).getStringValue();
@@ -99,8 +101,9 @@
String language = ((AString) functionRecord
.getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_LANGUAGE_FIELD_INDEX)).getStringValue();
- String functionKind = ((AString) functionRecord
- .getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_KIND_FIELD_INDEX)).getStringValue();
+ String functionKind =
+ ((AString) functionRecord.getValueByPos(MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_KIND_FIELD_INDEX))
+ .getStringValue();
return new Function(dataverseName, functionName, Integer.parseInt(arity), params, returnType, definition,
language, functionKind);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 5dee24e..bf27fc8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -76,15 +76,15 @@
public static final String INDEX_ISENFORCED_FIELD_NAME = "IsEnforced";
public static final String INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME = "SearchKeySourceIndicator";
- private OrderedListBuilder listBuilder = new OrderedListBuilder();
- private OrderedListBuilder primaryKeyListBuilder = new OrderedListBuilder();
- private AOrderedListType stringList = new AOrderedListType(BuiltinType.ASTRING, null);
- private AOrderedListType int8List = new AOrderedListType(BuiltinType.AINT8, null);
- private ArrayBackedValueStorage nameValue = new ArrayBackedValueStorage();
- private ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
- private List<List<String>> searchKey;
- private List<IAType> searchKeyType;
- private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+ private transient OrderedListBuilder listBuilder = new OrderedListBuilder();
+ private transient OrderedListBuilder primaryKeyListBuilder = new OrderedListBuilder();
+ private transient AOrderedListType stringList = new AOrderedListType(BuiltinType.ASTRING, null);
+ private transient AOrderedListType int8List = new AOrderedListType(BuiltinType.AINT8, null);
+ private transient ArrayBackedValueStorage nameValue = new ArrayBackedValueStorage();
+ private transient ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+ private transient List<List<String>> searchKey;
+ private transient List<IAType> searchKeyType;
+ private transient AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.AINT32);
@@ -97,7 +97,7 @@
private final MetadataNode metadataNode;
private final JobId jobId;
- public IndexTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
+ protected IndexTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.INDEX_DATASET.getFieldCount());
this.jobId = jobId;
this.metadataNode = metadataNode;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
index fb38638..d259212 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -52,7 +52,7 @@
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.LIBRARY_RECORDTYPE);
- public LibraryTupleTranslator(boolean getTuple) {
+ protected LibraryTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.LIBRARY_DATASET.getFieldCount());
}
@@ -63,15 +63,17 @@
int recordLength = frameTuple.getFieldLength(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
DataInput in = new DataInputStream(stream);
- ARecord libraryRecord = (ARecord) recordSerDes.deserialize(in);
+ ARecord libraryRecord = recordSerDes.deserialize(in);
return createLibraryFromARecord(libraryRecord);
}
private Library createLibraryFromARecord(ARecord libraryRecord) {
- String dataverseName = ((AString) libraryRecord
- .getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
- String libraryName = ((AString) libraryRecord
- .getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_NAME_FIELD_INDEX)).getStringValue();
+ String dataverseName =
+ ((AString) libraryRecord.getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX))
+ .getStringValue();
+ String libraryName =
+ ((AString) libraryRecord.getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_NAME_FIELD_INDEX))
+ .getStringValue();
return new Library(dataverseName, libraryName);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
new file mode 100644
index 0000000..20f04c4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
@@ -0,0 +1,78 @@
+/*
+ * 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.metadata.entitytupletranslators;
+
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.metadata.MetadataNode;
+
+public class MetadataTupleTranslatorProvider {
+
+ public CompactionPolicyTupleTranslator getCompactionPolicyTupleTranslator(boolean getTuple) {
+ return new CompactionPolicyTupleTranslator(getTuple);
+ }
+
+ public DatasetTupleTranslator getDatasetTupleTranslator(boolean getTuple) {
+ return new DatasetTupleTranslator(getTuple);
+ }
+
+ public DatasourceAdapterTupleTranslator getAdapterTupleTranslator(boolean getTuple) {
+ return new DatasourceAdapterTupleTranslator(getTuple);
+ }
+
+ public DatatypeTupleTranslator getDataTypeTupleTranslator(JobId jobId, MetadataNode metadataNode,
+ boolean getTuple) {
+ return new DatatypeTupleTranslator(jobId, metadataNode, getTuple);
+ }
+
+ public DataverseTupleTranslator getDataverseTupleTranslator(boolean getTuple) {
+ return new DataverseTupleTranslator(getTuple);
+ }
+
+ public ExternalFileTupleTranslator getExternalFileTupleTranslator(boolean getTuple) {
+ return new ExternalFileTupleTranslator(getTuple);
+ }
+
+ public FeedPolicyTupleTranslator getFeedPolicyTupleTranslator(boolean getTuple) {
+ return new FeedPolicyTupleTranslator(getTuple);
+ }
+
+ public FeedTupleTranslator getFeedTupleTranslator(boolean getTuple) {
+ return new FeedTupleTranslator(getTuple);
+ }
+
+ public FunctionTupleTranslator getFunctionTupleTranslator(boolean getTuple) {
+ return new FunctionTupleTranslator(getTuple);
+ }
+
+ public IndexTupleTranslator getIndexTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
+ return new IndexTupleTranslator(jobId, metadataNode, getTuple);
+ }
+
+ public LibraryTupleTranslator getLibraryTupleTranslator(boolean getTuple) {
+ return new LibraryTupleTranslator(getTuple);
+ }
+
+ public NodeTupleTranslator getNodeTupleTranslator(boolean getTuple) {
+ return new NodeTupleTranslator(getTuple);
+ }
+
+ public NodeGroupTupleTranslator getNodeGroupTupleTranslator(boolean getTuple) {
+ return new NodeGroupTupleTranslator(getTuple);
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
index af83ee9..397021d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
@@ -47,20 +47,21 @@
*/
public class NodeGroupTupleTranslator extends AbstractTupleTranslator<NodeGroup> {
+ private static final long serialVersionUID = 1L;
// Field indexes of serialized NodeGroup in a tuple.
// First key field.
public static final int NODEGROUP_NODEGROUPNAME_TUPLE_FIELD_INDEX = 0;
// Payload field containing serialized NodeGroup.
public static final int NODEGROUP_PAYLOAD_TUPLE_FIELD_INDEX = 1;
- private UnorderedListBuilder listBuilder = new UnorderedListBuilder();
- private ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+ private transient UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+ private transient ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
private List<String> nodeNames;
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.NODEGROUP_RECORDTYPE);
- public NodeGroupTupleTranslator(boolean getTuple) {
+ protected NodeGroupTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.NODEGROUP_DATASET.getFieldCount());
}
@@ -71,7 +72,7 @@
int recordLength = frameTuple.getFieldLength(NODEGROUP_PAYLOAD_TUPLE_FIELD_INDEX);
ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
DataInput in = new DataInputStream(stream);
- ARecord nodeGroupRecord = (ARecord) recordSerDes.deserialize(in);
+ ARecord nodeGroupRecord = recordSerDes.deserialize(in);
String gpName = ((AString) nodeGroupRecord
.getValueByPos(MetadataRecordTypes.NODEGROUP_ARECORD_GROUPNAME_FIELD_INDEX)).getStringValue();
IACursor cursor = ((AUnorderedList) nodeGroupRecord
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
index 55b61db..a54be65 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
@@ -44,7 +44,7 @@
// Payload field containing serialized Node.
public static final int NODE_PAYLOAD_TUPLE_FIELD_INDEX = 1;
- private AMutableInt64 aInt64 = new AMutableInt64(-1);
+ private transient AMutableInt64 aInt64 = new AMutableInt64(-1);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.AINT64);
@@ -54,7 +54,7 @@
// NonTaggedSerializerDeserializerProvider.INSTANCE
// .getSerializerDeserializer(recordType);
- public NodeTupleTranslator(boolean getTuple) {
+ protected NodeTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.NODE_DATASET.getFieldCount());
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
index 8ef6732..57284b0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
@@ -21,9 +21,9 @@
import java.util.HashMap;
import java.util.Map;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.utils.MetadataConstants;
public class BuiltinFeedPolicies {
@@ -152,7 +152,8 @@
policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "true");
String description = "AdvancedFT 100% Discard during congestion";
- return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Spill", description, policyParams);
+ return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Spill", description,
+ policyParams);
}
// AdvancedFT_Spill
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 12ab634..f91ab8e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -25,12 +25,10 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
-import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -59,6 +57,7 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.IAType;
@@ -81,10 +80,8 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.common.data.partition.RandomPartitionComputerFactory;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningWithMessageConnectorDescriptor;
@@ -96,11 +93,6 @@
private static final Logger LOGGER = Logger.getLogger(FeedMetadataUtil.class.getName());
- private static class LocationConstraint {
- int partition;
- String location;
- }
-
public static Dataset validateIfDatasetExists(String dataverse, String datasetName, MetadataTransactionContext ctx)
throws AsterixException {
Dataset dataset = MetadataManager.INSTANCE.getDataset(ctx, dataverse, datasetName);
@@ -128,8 +120,8 @@
MetadataTransactionContext ctx) throws AsterixException {
FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, dataverse, policyName);
if (feedPolicy == null) {
- feedPolicy =
- MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME, policyName);
+ feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ policyName);
if (feedPolicy == null) {
throw new AsterixException("Unknown feed policy" + policyName);
}
@@ -170,11 +162,11 @@
boolean enableSubscriptionMode;
OperatorDescriptorId opId = null;
if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
- IPushRuntimeFactory[] runtimeFactories =
- ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline().getRuntimeFactories();
+ IPushRuntimeFactory[] runtimeFactories = ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline()
+ .getRuntimeFactories();
if (runtimeFactories[0] instanceof AssignRuntimeFactory && runtimeFactories.length > 1) {
- IConnectorDescriptor connectorDesc =
- spec.getOperatorInputMap().get(opDesc.getOperatorId()).get(0);
+ IConnectorDescriptor connectorDesc = spec.getOperatorInputMap().get(opDesc.getOperatorId())
+ .get(0);
IOperatorDescriptor sourceOp = spec.getProducer(connectorDesc);
if (sourceOp instanceof FeedCollectOperatorDescriptor) {
runtimeType = FeedRuntimeType.COMPUTE;
@@ -209,16 +201,16 @@
}
// make connections between operators
- for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>
- entry : spec.getConnectorOperatorMap().entrySet()) {
+ for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>,
+ Pair<IOperatorDescriptor, Integer>>> entry : spec.getConnectorOperatorMap().entrySet()) {
IConnectorDescriptor connDesc = altered.getConnectorMap().get(connectorMapping.get(entry.getKey()));
Pair<IOperatorDescriptor, Integer> leftOp = entry.getValue().getLeft();
Pair<IOperatorDescriptor, Integer> rightOp = entry.getValue().getRight();
- IOperatorDescriptor leftOpDesc =
- altered.getOperatorMap().get(oldNewOID.get(leftOp.getLeft().getOperatorId()));
- IOperatorDescriptor rightOpDesc =
- altered.getOperatorMap().get(oldNewOID.get(rightOp.getLeft().getOperatorId()));
+ IOperatorDescriptor leftOpDesc = altered.getOperatorMap()
+ .get(oldNewOID.get(leftOp.getLeft().getOperatorId()));
+ IOperatorDescriptor rightOpDesc = altered.getOperatorMap()
+ .get(oldNewOID.get(rightOp.getLeft().getOperatorId()));
altered.connect(connDesc, leftOpDesc, leftOp.getRight(), rightOpDesc, rightOp.getRight());
}
@@ -246,9 +238,8 @@
operatorLocations.put(opDesc.getOperatorId(), locations);
}
String location = (String) ((ConstantExpression) cexpr).getValue();
- LocationConstraint lc = new LocationConstraint();
- lc.location = location;
- lc.partition = ((PartitionLocationExpression) lexpr).getPartition();
+ LocationConstraint lc = new LocationConstraint(location,
+ ((PartitionLocationExpression) lexpr).getPartition());
locations.add(lc);
break;
default:
@@ -299,125 +290,6 @@
}
- public static void increaseCardinality(JobSpecification spec, FeedRuntimeType compute, int requiredCardinality,
- List<String> newLocations) throws AsterixException {
- IOperatorDescriptor changingOpDesc = alterJobSpecForComputeCardinality(spec, requiredCardinality);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, changingOpDesc,
- nChooseK(requiredCardinality, newLocations));
-
- }
-
- public static void decreaseComputeCardinality(JobSpecification spec, FeedRuntimeType compute,
- int requiredCardinality, List<String> currentLocations) throws AsterixException {
- IOperatorDescriptor changingOpDesc = alterJobSpecForComputeCardinality(spec, requiredCardinality);
- String[] chosenLocations = nChooseK(requiredCardinality, currentLocations);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, changingOpDesc, chosenLocations);
- }
-
- private static IOperatorDescriptor alterJobSpecForComputeCardinality(
- JobSpecification spec, int requiredCardinality) throws AsterixException {
- Map<ConnectorDescriptorId, IConnectorDescriptor> connectors = spec.getConnectorMap();
- Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>
- connectorOpMap = spec.getConnectorOperatorMap();
- IOperatorDescriptor sourceOp = null;
- IOperatorDescriptor targetOp = null;
- IConnectorDescriptor connDesc = null;
- for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>
- entry : connectorOpMap.entrySet()) {
- ConnectorDescriptorId cid = entry.getKey();
- sourceOp = entry.getValue().getKey().getKey();
- if (sourceOp instanceof FeedCollectOperatorDescriptor) {
- targetOp = entry.getValue().getValue().getKey();
- if ((targetOp instanceof FeedMetaOperatorDescriptor)
- && (((FeedMetaOperatorDescriptor) targetOp).getRuntimeType()
- .equals(FeedRuntimeType.COMPUTE))) {
- connDesc = connectors.get(cid);
- break;
- } else {
- throw new AsterixException("Incorrect manipulation, feed does not have a compute stage");
- }
- }
- }
-
- Map<OperatorDescriptorId, List<IConnectorDescriptor>> operatorInputMap = spec.getOperatorInputMap();
- boolean removed = operatorInputMap.get(targetOp.getOperatorId()).remove(connDesc);
- if (!removed) {
- throw new AsterixException("Connector desc not found");
- }
- Map<OperatorDescriptorId, List<IConnectorDescriptor>> operatorOutputMap = spec.getOperatorOutputMap();
- removed = operatorOutputMap.get(sourceOp.getOperatorId()).remove(connDesc);
- if (!removed) {
- throw new AsterixException("Connector desc not found");
- }
- spec.getConnectorMap().remove(connDesc.getConnectorId());
- connectorOpMap.remove(connDesc.getConnectorId());
-
- ITuplePartitionComputerFactory tpcf = new RandomPartitionComputerFactory();
- MToNPartitioningConnectorDescriptor newConnector = new MToNPartitioningConnectorDescriptor(spec, tpcf);
- spec.getConnectorMap().put(newConnector.getConnectorId(), newConnector);
- spec.connect(newConnector, sourceOp, 0, targetOp, 0);
-
- // ==============================================================================
- Set<Constraint> userConstraints = spec.getUserConstraints();
- Constraint countConstraint = null;
- Constraint locationConstraint = null;
- List<LocationConstraint> locations = new ArrayList<LocationConstraint>();
- IOperatorDescriptor changingOpDesc = null;
-
- for (Constraint constraint : userConstraints) {
- LValueConstraintExpression lexpr = constraint.getLValue();
- ConstraintExpression cexpr = constraint.getRValue();
- OperatorDescriptorId opId;
- switch (lexpr.getTag()) {
- case PARTITION_COUNT: {
- opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
- IOperatorDescriptor opDesc = spec.getOperatorMap().get(opId);
- if (opDesc instanceof FeedMetaOperatorDescriptor) {
- FeedRuntimeType runtimeType = ((FeedMetaOperatorDescriptor) opDesc).getRuntimeType();
- if (runtimeType.equals(FeedRuntimeType.COMPUTE)) {
- countConstraint = constraint;
- changingOpDesc = opDesc;
- }
- }
- break;
- }
- case PARTITION_LOCATION:
- opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
- IOperatorDescriptor opDesc = spec.getOperatorMap().get(opId);
- if (opDesc instanceof FeedMetaOperatorDescriptor) {
- FeedRuntimeType runtimeType = ((FeedMetaOperatorDescriptor) opDesc).getRuntimeType();
- if (runtimeType.equals(FeedRuntimeType.COMPUTE)) {
- locationConstraint = constraint;
- changingOpDesc = opDesc;
- String location = (String) ((ConstantExpression) cexpr).getValue();
- LocationConstraint lc = new LocationConstraint();
- lc.location = location;
- lc.partition = ((PartitionLocationExpression) lexpr).getPartition();
- locations.add(lc);
- }
- }
- break;
- default:
- break;
- }
- }
-
- userConstraints.remove(countConstraint);
- if (locationConstraint != null) {
- userConstraints.remove(locationConstraint);
- }
-
- return changingOpDesc;
- }
-
- private static String[] nChooseK(int k, List<String> locations) {
- String[] result = new String[k];
- for (int i = 0; i < k; i++) {
- result[i] = locations.get(i);
- }
- return result;
- }
-
private static boolean preProcessingRequired(FeedConnectionId connectionId) {
MetadataTransactionContext ctx = null;
Feed feed = null;
@@ -584,8 +456,7 @@
if (ExternalDataUtils.isChangeFeed(configuration)) {
getSerdesForPKs(serdes, configuration, metaType, adapterOutputType, i);
}
- feedProps = new Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType>(adapterFactory,
- new RecordDescriptor(serdes), adapterType);
+ feedProps = new Triple<>(adapterFactory, new RecordDescriptor(serdes), adapterType);
} catch (Exception e) {
throw new AlgebricksException("unable to create adapter", e);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/LocationConstraint.java
similarity index 73%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/LocationConstraint.java
index 1752054..24d6c5a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/LocationConstraint.java
@@ -16,12 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.metadata.feeds;
-import java.util.Map;
+public class LocationConstraint {
+ public final String location;
+ public final int partition;
-public interface ISubscriberRuntime {
-
- public Map<String, String> getFeedPolicy();
-
-}
+ public LocationConstraint(String location, int partition) {
+ this.location = location;
+ this.partition = partition;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
index d0af744..56439b6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
@@ -26,7 +26,6 @@
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -257,8 +256,8 @@
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
propertyRecordBuilder.reset(recordType);
AMutableString aString = new AMutableString("");
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AString> stringSerde =
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
// write field 0
fieldValue.reset();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
similarity index 85%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataConstants.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index 943e385..fbd2380 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -17,15 +17,17 @@
* under the License.
*/
-package org.apache.asterix.common.config;
+package org.apache.asterix.metadata.utils;
public class MetadataConstants {
+ private MetadataConstants() {
+ }
// Name of the dataverse the metadata lives in.
- public final static String METADATA_DATAVERSE_NAME = "Metadata";
+ public static final String METADATA_DATAVERSE_NAME = "Metadata";
// Name of the node group where metadata is stored on.
- public final static String METADATA_NODEGROUP_NAME = "MetadataGroup";
+ public static final String METADATA_NODEGROUP_NAME = "MetadataGroup";
// Name of the default nodegroup where internal/feed datasets will be partitioned
// if an explicit nodegroup is not specified at the time of creation of a dataset
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
index 601ce15..a31f2a6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
@@ -23,7 +23,6 @@
import java.util.List;
import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataConstants;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.MetadataManager;
@@ -38,16 +37,19 @@
public class SplitsAndConstraintsUtil {
+ private SplitsAndConstraintsUtil() {
+ }
+
private static FileSplit[] splitsForDataverse(String dataverseName) {
File relPathFile = new File(dataverseName);
- List<FileSplit> splits = new ArrayList<FileSplit>();
+ List<FileSplit> splits = new ArrayList<>();
// get all partitions
ClusterPartition[] clusterPartition = AsterixClusterProperties.INSTANCE.getClusterPartitons();
String storageDirName = AsterixClusterProperties.INSTANCE.getStorageDirectoryName();
for (int j = 0; j < clusterPartition.length; j++) {
- int nodeParitions = AsterixClusterProperties.INSTANCE
- .getNodePartitionsCount(clusterPartition[j].getNodeId());
- for (int i = 0; i < nodeParitions; i++) {
+ int nodePartitions =
+ AsterixClusterProperties.INSTANCE.getNodePartitionsCount(clusterPartition[j].getNodeId());
+ for (int i = 0; i < nodePartitions; i++) {
File f = new File(StoragePathUtil.prepareStoragePartitionPath(storageDirName,
clusterPartition[i].getPartitionId()) + File.separator + relPathFile);
splits.add(StoragePathUtil.getFileSplitForClusterPartition(clusterPartition[j], f));
@@ -59,17 +61,17 @@
public static FileSplit[] splitsForDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
String datasetName, String targetIdxName, boolean temp) throws AlgebricksException {
try {
- File relPathFile = new File(
- StoragePathUtil.prepareDataverseIndexName(dataverseName, datasetName, targetIdxName));
+ File relPathFile =
+ new File(StoragePathUtil.prepareDataverseIndexName(dataverseName, datasetName, targetIdxName));
Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
- List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName())
- .getNodeNames();
+ List<String> nodeGroup =
+ MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName()).getNodeNames();
if (nodeGroup == null) {
throw new AlgebricksException("Couldn't find node group " + dataset.getNodeGroupName());
}
String storageDirName = AsterixClusterProperties.INSTANCE.getStorageDirectoryName();
- List<FileSplit> splits = new ArrayList<FileSplit>();
+ List<FileSplit> splits = new ArrayList<>();
for (String nd : nodeGroup) {
int numPartitions = AsterixClusterProperties.INSTANCE.getNodePartitionsCount(nd);
ClusterPartition[] nodePartitions = AsterixClusterProperties.INSTANCE.getNodePartitions(nd);
@@ -96,16 +98,16 @@
private static FileSplit[] splitsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
try {
- File relPathFile = new File(
- StoragePathUtil.prepareDataverseIndexName(dataverseName, datasetName, targetIdxName));
+ File relPathFile =
+ new File(StoragePathUtil.prepareDataverseIndexName(dataverseName, datasetName, targetIdxName));
Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
- List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName())
- .getNodeNames();
+ List<String> nodeGroup =
+ MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName()).getNodeNames();
if (nodeGroup == null) {
throw new AlgebricksException("Couldn't find node group " + dataset.getNodeGroupName());
}
- List<FileSplit> splits = new ArrayList<FileSplit>();
+ List<FileSplit> splits = new ArrayList<>();
for (String nodeId : nodeGroup) {
// get node partitions
ClusterPartition[] nodePartitions = AsterixClusterProperties.INSTANCE.getNodePartitions(nodeId);
@@ -130,15 +132,15 @@
}
}
- public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
- String dataverse) {
+ public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
+ splitProviderAndPartitionConstraintsForDataverse(String dataverse) {
FileSplit[] splits = splitsForDataverse(dataverse);
return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
}
- public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForFilesIndex(
- MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName, String targetIdxName,
- boolean create) throws AlgebricksException {
+ public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
+ splitProviderAndPartitionConstraintsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
+ String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
FileSplit[] splits = splitsForFilesIndex(mdTxnCtx, dataverseName, datasetName, targetIdxName, create);
return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
}
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index 2671834..6f3bd30 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -25,7 +25,6 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.metadata.MetadataException;
-import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
@@ -51,11 +50,9 @@
indicator == null ? null : Collections.singletonList(indicator),
Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList(), false);
- Dataset dataset = new Dataset("test", "log", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES",
- "prefix", compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115,
- 0);
-
- MetadataPrimaryIndexes.init();
+ Dataset dataset =
+ new Dataset("test", "log", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES", "prefix",
+ compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0);
DatasetTupleTranslator dtTranslator = new DatasetTupleTranslator(true);
ITupleReference tuple = dtTranslator.getTupleFromMetadataEntity(dataset);
Dataset deserializedDataset = dtTranslator.getMetadataEntityFromTuple(tuple);
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index 02b3e4b..46bacdd 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -32,7 +32,6 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.MetadataNode;
-import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Datatype;
import org.apache.asterix.metadata.entities.Index;
@@ -62,17 +61,15 @@
indicator == null ? null : Collections.singletonList(indicator),
Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList(), false);
- Dataset dataset = new Dataset("test", "d1", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES",
- "prefix", compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115,
- 0);
+ Dataset dataset =
+ new Dataset("test", "d1", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES", "prefix",
+ compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0);
Index index = new Index("test", "d1", "i1", IndexType.BTREE,
Collections.singletonList(Collections.singletonList("row_id")),
indicator == null ? null : Collections.singletonList(indicator),
Collections.singletonList(BuiltinType.AINT64), -1, false, false, 0);
- MetadataPrimaryIndexes.init();
-
MetadataNode mockMetadataNode = mock(MetadataNode.class);
when(mockMetadataNode.getDatatype(any(), anyString(), anyString())).thenReturn(new Datatype("test", "d1",
new ARecordType("", new String[] { "row_id" }, new IAType[] { BuiltinType.AINT64 }, true), true));
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java
index 4034d77..a1a89fe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java
@@ -18,15 +18,16 @@
*/
package org.apache.asterix.om.base;
-import org.json.JSONException;
-import org.json.JSONObject;
+import java.io.Serializable;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.visitors.IOMVisitor;
+import org.json.JSONException;
+import org.json.JSONObject;
-public class AString implements IAObject {
+public class AString implements IAObject, Serializable {
protected String value;
@@ -50,8 +51,9 @@
@Override
public boolean equals(Object obj) {
- if (!(obj instanceof AString))
+ if (!(obj instanceof AString)) {
return false;
+ }
return value.equals(((AString) obj).getStringValue());
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index 248ec3a..fc1108f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -44,8 +44,8 @@
*/
public class ARecordType extends AbstractComplexType {
- public static final ARecordType FULLY_OPEN_RECORD_TYPE =
- new ARecordType("OpenRecord", new String[0], new IAType[0], true);
+ public static final ARecordType FULLY_OPEN_RECORD_TYPE = new ARecordType("OpenRecord", new String[0], new IAType[0],
+ true);
private static final long serialVersionUID = 1L;
private final String[] fieldNames;
@@ -369,4 +369,14 @@
}
return false;
}
+
+ /**
+ * Create a fully open record type with the passed name
+ *
+ * @param name
+ * @return
+ */
+ public static ARecordType createOpenRecordType(String name) {
+ return new ARecordType(name, new String[0], new IAType[0], true);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java
index bb1e554..bf103fc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java
@@ -18,11 +18,13 @@
*/
package org.apache.asterix.om.util;
+import java.io.IOException;
import java.util.logging.Logger;
import org.apache.asterix.common.cluster.IGlobalRecoveryMaanger;
import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixCompilerProperties;
+import org.apache.asterix.common.config.AsterixExtensionProperties;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.AsterixFeedProperties;
import org.apache.asterix.common.config.AsterixMetadataProperties;
@@ -59,12 +61,15 @@
private AsterixFeedProperties feedProperties;
private AsterixBuildProperties buildProperties;
private AsterixReplicationProperties replicationProperties;
+ private AsterixExtensionProperties extensionProperties;
private final IGlobalRecoveryMaanger globalRecoveryMaanger;
private IHyracksClientConnection hcc;
private final ILibraryManager libraryManager;
+ private Object extensionManager;
public static void initialize(ICCApplicationContext ccAppCtx, IHyracksClientConnection hcc,
- IGlobalRecoveryMaanger globalRecoveryMaanger, ILibraryManager libraryManager) throws AsterixException {
+ IGlobalRecoveryMaanger globalRecoveryMaanger, ILibraryManager libraryManager)
+ throws AsterixException, IOException {
if (INSTANCE != null) {
return;
}
@@ -86,8 +91,9 @@
INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
INSTANCE.feedProperties = new AsterixFeedProperties(propertiesAccessor);
- INSTANCE.replicationProperties = new AsterixReplicationProperties(propertiesAccessor,
- AsterixClusterProperties.INSTANCE.getCluster());
+ INSTANCE.extensionProperties = new AsterixExtensionProperties(propertiesAccessor);
+ INSTANCE.replicationProperties =
+ new AsterixReplicationProperties(propertiesAccessor, AsterixClusterProperties.INSTANCE.getCluster());
INSTANCE.hcc = hcc;
INSTANCE.buildProperties = new AsterixBuildProperties(propertiesAccessor);
Logger.getLogger("org.apache").setLevel(INSTANCE.externalProperties.getLogLevel());
@@ -173,4 +179,16 @@
public ILibraryManager getLibraryManager() {
return libraryManager;
}
+
+ public Object getExtensionManager() {
+ return extensionManager;
+ }
+
+ public void setExtensionManager(Object extensionManager) {
+ this.extensionManager = extensionManager;
+ }
+
+ public AsterixExtensionProperties getExtensionProperties() {
+ return extensionProperties;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.java
new file mode 100644
index 0000000..2d23058
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.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.asterix.runtime.job.listener;
+
+import java.util.List;
+
+import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.hyracks.api.context.IHyracksJobletContext;
+import org.apache.hyracks.api.job.IJobletEventListener;
+import org.apache.hyracks.api.job.IJobletEventListenerFactory;
+import org.apache.hyracks.api.job.JobStatus;
+
+/**
+ * This Joblet enable transactions on multiple datasets to take place in the same Hyracks Job
+ * It takes a list of Transaction job ids instead of a single job Id
+ */
+public class MultiTransactionJobletEventListenerFactory implements IJobletEventListenerFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final List<JobId> jobIds;
+ private final boolean transactionalWrite;
+
+ public MultiTransactionJobletEventListenerFactory(List<JobId> jobIds, boolean transactionalWrite) {
+ this.jobIds = jobIds;
+ this.transactionalWrite = transactionalWrite;
+ }
+
+ @Override
+ public IJobletEventListener createListener(final IHyracksJobletContext jobletContext) {
+
+ return new IJobletEventListener() {
+ @Override
+ public void jobletFinish(JobStatus jobStatus) {
+ try {
+ ITransactionManager txnManager =
+ ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+ .getTransactionSubsystem().getTransactionManager();
+ for (JobId jobId : jobIds) {
+ ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
+ txnContext.setWriteTxn(transactionalWrite);
+ txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
+ !(jobStatus == JobStatus.FAILURE));
+ }
+ } catch (ACIDException e) {
+ throw new Error(e);
+ }
+ }
+
+ @Override
+ public void jobletStart() {
+ try {
+ for (JobId jobId : jobIds) {
+ ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+ .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
+ }
+ } catch (ACIDException e) {
+ throw new Error(e);
+ }
+ }
+
+ };
+ }
+}
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
index 696726f..003137d 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.server.test;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.logging.Logger;
+
import org.apache.asterix.test.aql.TestExecutor;
import org.apache.asterix.test.runtime.HDFSCluster;
import org.apache.asterix.testframework.context.TestCaseContext;
@@ -34,11 +39,6 @@
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.logging.Logger;
-
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
@RunWith(Parameterized.class)
public class NCServiceExecutionIT {
@@ -47,7 +47,7 @@
// The "target" subdirectory of asterix-server. All outputs go here.
private static final String TARGET_DIR = StringUtils
- .join(new String[] { System.getProperty("basedir"), "target" }, File.separator);
+ .join(new String[] { "target" }, File.separator);
// Directory where the NCs create and store all data, as configured by
// src/test/resources/NCServiceExecutionIT/cc.conf.
@@ -75,7 +75,7 @@
// paths in "load" statements in test queries to find the right data. It is
// also used for HDFSCluster.
private static final String ASTERIX_APP_DIR = StringUtils
- .join(new String[] { System.getProperty("basedir"), "..", "asterix-app" },
+ .join(new String[] { "..", "asterix-app" },
File.separator);
// Path to the actual AQL test files, which we borrow from asterix-app. This is
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index a03bc30..7285a5a 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -356,6 +356,19 @@
<ignore />
</action>
</pluginExecution>
+ <pluginExecution>
+ <pluginExecutionFilter>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>javacc-maven-plugin</artifactId>
+ <versionRange>[2.6,)</versionRange>
+ <goals>
+ <goal>jjdoc</goal>
+ </goals>
+ </pluginExecutionFilter>
+ <action>
+ <ignore></ignore>
+ </action>
+ </pluginExecution>
</pluginExecutions>
</lifecycleMappingMetadata>
</configuration>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java
index 2f2b7e3..afcb86e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java
@@ -20,7 +20,7 @@
import java.io.Serializable;
-public final class FunctionIdentifier implements Serializable {
+public class FunctionIdentifier implements Serializable {
private static final long serialVersionUID = 1L;
private final String namespace;
@@ -60,6 +60,7 @@
return name.hashCode() + namespace.hashCode();
}
+ @Override
public String toString() {
return getNamespace() + ":" + name;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IApplicationConfig.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IApplicationConfig.java
index bd40813..278a9d7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IApplicationConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IApplicationConfig.java
@@ -18,6 +18,8 @@
*/
package org.apache.hyracks.api.application;
+import java.util.List;
+import java.util.Map.Entry;
import java.util.Set;
/**
@@ -25,11 +27,22 @@
*/
public interface IApplicationConfig {
String getString(String section, String key);
+
String getString(String section, String key, String defaultValue);
+
int getInt(String section, String key);
+
int getInt(String section, String key, int defaultValue);
+
long getLong(String section, String key);
+
long getLong(String section, String key, long defaultValue);
+
Set<String> getSections();
+
Set<String> getKeys(String section);
+
+ String[] getStringArray(String section, String key);
+
+ List<Set<Entry<String, String>>> getMultiSections(String section);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ActivityId.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ActivityId.java
index acac345..214627e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ActivityId.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ActivityId.java
@@ -49,13 +49,17 @@
return odId;
}
+ public void setOperatorDescriptorId(OperatorDescriptorId odId) {
+ this.odId = odId;
+ }
+
public int getLocalId() {
return id;
}
@Override
public int hashCode() {
- return (int) (odId.hashCode() + id);
+ return odId.hashCode() + id;
}
@Override
@@ -70,6 +74,7 @@
return other.odId.equals(odId) && other.id == id;
}
+ @Override
public String toString() {
return "ANID:" + odId + ":" + id;
}
@@ -78,8 +83,8 @@
if (str.startsWith("ANID:")) {
str = str.substring(5);
int idIdx = str.lastIndexOf(':');
- return new ActivityId(OperatorDescriptorId.parse(str.substring(0, idIdx)), Integer.parseInt(str
- .substring(idIdx + 1)));
+ return new ActivityId(OperatorDescriptorId.parse(str.substring(0, idIdx)),
+ Integer.parseInt(str.substring(idIdx + 1)));
}
throw new IllegalArgumentException("Unable to parse: " + str);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IConnectorDescriptor.java
index 339eb9d..7219040 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -68,7 +68,7 @@
*/
public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException;
+ throws HyracksDataException;
/**
* Factory metod to create the receive side reader that reads data from this
@@ -136,4 +136,9 @@
* @throws JSONException
*/
public JSONObject toJSON() throws JSONException;
+
+ /**
+ * Sets the connector Id
+ */
+ public void setConnectorId(ConnectorDescriptorId cdId);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
index 1b85fbd..26561e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -20,12 +20,11 @@
import java.io.Serializable;
-import org.json.JSONException;
-import org.json.JSONObject;
-
import org.apache.hyracks.api.application.ICCApplicationContext;
import org.apache.hyracks.api.constraints.IConstraintAcceptor;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.json.JSONException;
+import org.json.JSONObject;
/**
* Descriptor for operators in Hyracks.
@@ -41,6 +40,13 @@
public OperatorDescriptorId getOperatorId();
/**
+ * Sets the id of the operator.
+ *
+ * @param id
+ */
+ void setOperatorId(OperatorDescriptorId id);
+
+ /**
* Returns the number of inputs into this operator.
*
* @return Number of inputs.
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index e4385bf..c95c31c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -26,6 +26,7 @@
public static final int ERROR_PROCESSING_TUPLE = 0;
public static final int INVALID_OPERATOR_OPERATION = 1;
public static final int FAILURE_ON_NODE = 2;
+ public static final int ILLEGAL_ARGUMENT = 3;
private ErrorCode() {
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
index 5165a62..7b44ff5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
@@ -28,10 +28,6 @@
import java.util.Set;
import org.apache.commons.lang3.tuple.Pair;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
-
import org.apache.hyracks.api.constraints.Constraint;
import org.apache.hyracks.api.constraints.expressions.ConstantExpression;
import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
@@ -43,6 +39,9 @@
import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.dataset.ResultSetId;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
private static final long serialVersionUID = 1L;
@@ -98,7 +97,7 @@
connMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
opInputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
opOutputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
- connectorOpMap = new HashMap<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>();
+ connectorOpMap = new HashMap<>();
properties = new HashMap<String, Serializable>();
userConstraints = new HashSet<Constraint>();
operatorIdCounter = 0;
@@ -112,6 +111,7 @@
@Override
public OperatorDescriptorId createOperatorDescriptorId(IOperatorDescriptor op) {
OperatorDescriptorId odId = new OperatorDescriptorId(operatorIdCounter++);
+ op.setOperatorId(odId);
opMap.put(odId, op);
return odId;
}
@@ -119,6 +119,7 @@
@Override
public ConnectorDescriptorId createConnectorDescriptor(IConnectorDescriptor conn) {
ConnectorDescriptorId cdId = new ConnectorDescriptorId(connectorIdCounter++);
+ conn.setConnectorId(cdId);
connMap.put(cdId, conn);
return cdId;
}
@@ -135,8 +136,7 @@
IOperatorDescriptor consumerOp, int consumerPort) {
insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
insertIntoIndexedMap(opOutputMap, producerOp.getOperatorId(), producerPort, conn);
- connectorOpMap.put(
- conn.getConnectorId(),
+ connectorOpMap.put(conn.getConnectorId(),
Pair.<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> of(
Pair.<IOperatorDescriptor, Integer> of(producerOp, producerPort),
Pair.<IOperatorDescriptor, Integer> of(consumerOp, consumerPort)));
@@ -166,20 +166,20 @@
}
public RecordDescriptor getConnectorRecordDescriptor(IConnectorDescriptor conn) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
- .getConnectorId());
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap
+ .get(conn.getConnectorId());
return connInfo.getLeft().getLeft().getOutputRecordDescriptors()[connInfo.getLeft().getRight()];
}
public IOperatorDescriptor getConsumer(IConnectorDescriptor conn) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
- .getConnectorId());
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap
+ .get(conn.getConnectorId());
return connInfo.getRight().getLeft();
}
public int getConsumerInputIndex(IConnectorDescriptor conn) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
- .getConnectorId());
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap
+ .get(conn.getConnectorId());
return connInfo.getRight().getRight();
}
@@ -220,14 +220,14 @@
}
public IOperatorDescriptor getProducer(IConnectorDescriptor conn) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
- .getConnectorId());
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap
+ .get(conn.getConnectorId());
return connInfo.getLeft().getLeft();
}
public int getProducerOutputIndex(IConnectorDescriptor conn) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
- .getConnectorId());
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap
+ .get(conn.getConnectorId());
return connInfo.getLeft().getRight();
}
@@ -313,6 +313,7 @@
vList.set(index, value);
}
+ @Override
public String toString() {
StringBuilder buffer = new StringBuilder();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/IniApplicationConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/IniApplicationConfig.java
index 22fe318..53db11c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/IniApplicationConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/IniApplicationConfig.java
@@ -18,11 +18,15 @@
*/
package org.apache.hyracks.control.common.application;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
import org.apache.hyracks.api.application.IApplicationConfig;
import org.apache.hyracks.control.common.controllers.IniUtils;
import org.ini4j.Ini;
-
-import java.util.Set;
+import org.ini4j.Profile.Section;
/**
* An implementation of IApplicationConfig which is backed by Ini4j.
@@ -49,6 +53,11 @@
}
@Override
+ public String[] getStringArray(String section, String key) {
+ return IniUtils.getStringArray(ini, section, key);
+ }
+
+ @Override
public int getInt(String section, String key) {
return IniUtils.getInt(ini, section, key, 0);
}
@@ -60,7 +69,7 @@
@Override
public long getLong(String section, String key) {
- return IniUtils.getLong(ini, section, key, (long) 0);
+ return IniUtils.getLong(ini, section, key, 0);
}
@Override
@@ -77,4 +86,20 @@
public Set<String> getKeys(String section) {
return ini.get(section).keySet();
}
+
+ @Override
+ public List<Set<Map.Entry<String, String>>> getMultiSections(String section) {
+ List<Set<Map.Entry<String, String>>> list = new ArrayList<>();
+ List<Section> secs = getMulti(section);
+ if (secs != null) {
+ for (Section sec : secs) {
+ list.add(sec.entrySet());
+ }
+ }
+ return list;
+ }
+
+ private List<Section> getMulti(String section) {
+ return ini.getAll(section);
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/IniUtils.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/IniUtils.java
index 538bb0b..c6c3e73 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/IniUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/IniUtils.java
@@ -18,11 +18,13 @@
*/
package org.apache.hyracks.control.common.controllers;
-import org.ini4j.Ini;
-
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.lang.reflect.Array;
+
+import org.ini4j.Ini;
+import org.ini4j.Profile.Section;
/**
* Some utility functions for reading Ini4j objects with default values.
@@ -49,10 +51,27 @@
return (value != null) ? value : default_value;
}
+ @SuppressWarnings("unchecked")
+ private static <T> T getIniArray(Ini ini, String section, String key, Class<T> clazz) {
+ Section sec = ini.get(section);
+ if (clazz.getComponentType() == null) {
+ return null;
+ }
+ if (sec == null) {
+ return (T) Array.newInstance(clazz.getComponentType(), 0);
+ } else {
+ return sec.getAll(key, clazz);
+ }
+ }
+
public static String getString(Ini ini, String section, String key, String defaultValue) {
return getIniValue(ini, section, key, defaultValue, String.class);
}
+ public static String[] getStringArray(Ini ini, String section, String key) {
+ return getIniArray(ini, section, key, String[].class);
+ }
+
public static int getInt(Ini ini, String section, String key, int defaultValue) {
return getIniValue(ini, section, key, defaultValue, Integer.class);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index 5caf477..e780ea0 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -18,19 +18,18 @@
*/
package org.apache.hyracks.dataflow.std.base;
-import org.json.JSONException;
-import org.json.JSONObject;
-
import org.apache.hyracks.api.application.ICCApplicationContext;
import org.apache.hyracks.api.constraints.IConstraintAcceptor;
import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.job.ActivityCluster;
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.json.JSONException;
+import org.json.JSONObject;
public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
private static final long serialVersionUID = 1L;
- protected final ConnectorDescriptorId id;
+ protected ConnectorDescriptorId id;
protected String displayName;
@@ -39,14 +38,17 @@
displayName = getClass().getName() + "[" + id + "]";
}
+ @Override
public ConnectorDescriptorId getConnectorId() {
return id;
}
+ @Override
public String getDisplayName() {
return displayName;
}
+ @Override
public void setDisplayName(String displayName) {
this.displayName = displayName;
}
@@ -67,4 +69,9 @@
ICCApplicationContext appCtx) {
// do nothing
}
+
+ @Override
+ public void setConnectorId(ConnectorDescriptorId id) {
+ this.id = id;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index e930a2b..4f22a17 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -18,20 +18,19 @@
*/
package org.apache.hyracks.dataflow.std.base;
-import org.json.JSONException;
-import org.json.JSONObject;
-
import org.apache.hyracks.api.application.ICCApplicationContext;
import org.apache.hyracks.api.constraints.IConstraintAcceptor;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.json.JSONException;
+import org.json.JSONObject;
public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
private static final long serialVersionUID = 1L;
- protected final OperatorDescriptorId odId;
+ protected OperatorDescriptorId odId;
protected String[] partitions;
@@ -57,6 +56,11 @@
}
@Override
+ public void setOperatorId(OperatorDescriptorId id) {
+ this.odId = id;
+ }
+
+ @Override
public int getInputArity() {
return inputArity;
}
@@ -71,10 +75,12 @@
return recordDescriptors;
}
+ @Override
public String getDisplayName() {
return displayName;
}
+ @Override
public void setDisplayName(String displayName) {
this.displayName = displayName;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
index 775c367..9437b00 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.ActivityId;
import org.apache.hyracks.api.dataflow.IActivity;
import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements IActivity {
@@ -39,6 +40,14 @@
}
@Override
+ public final void setOperatorId(OperatorDescriptorId id) {
+ super.setOperatorId(id);
+ if (activityNodeId != null && !activityNodeId.getOperatorDescriptorId().equals(odId)) {
+ activityNodeId.setOperatorDescriptorId(odId);
+ }
+ }
+
+ @Override
public final void contributeActivities(IActivityGraphBuilder builder) {
builder.addActivity(this, this);
for (int i = 0; i < getInputArity(); ++i) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
index 44d77ac..b1cd83e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
@@ -60,7 +60,7 @@
@Override
public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(),
nConsumerPartitions, localityMap, index);
}
@@ -78,8 +78,9 @@
int receiverIndex, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
BitSet expectedPartitions = new BitSet(nProducerPartitions);
for (int i = 0; i < nProducerPartitions; i++) {
- if (localityMap.isConnected(i, receiverIndex, nConsumerPartitions))
+ if (localityMap.isConnected(i, receiverIndex, nConsumerPartitions)) {
expectedPartitions.set(i);
+ }
}
NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions,
expectedPartitions);
@@ -87,5 +88,4 @@
return new PartitionCollector(ctx, getConnectorId(), receiverIndex, expectedPartitions, frameReader,
channelReader);
}
-
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
index 2ca70da..d748c8e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
@@ -34,16 +34,17 @@
import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
public class MToNBroadcastConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
public MToNBroadcastConnectorDescriptor(IConnectorDescriptorRegistry spec) {
super(spec);
}
- private static final long serialVersionUID = 1L;
-
@Override
public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
final boolean[] isOpen = new boolean[nConsumerPartitions];
for (int i = 0; i < nConsumerPartitions; ++i) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
index 4872b95..d26b9ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
@@ -45,7 +45,7 @@
@Override
public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner());
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
index 04de894..edcad42 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -69,14 +69,14 @@
public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
throws HyracksDataException {
- final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
- recordDesc, tpcf.createPartitioner());
+ final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
+ tpcf.createPartitioner());
return hashWriter;
}
@Override
- public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
- int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+ int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
index e0c886f..f773918 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -48,7 +48,7 @@
@Override
public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
return edwFactory.createFrameWriter(index);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index 56af78e..67af861 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -80,16 +80,16 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
- SplitterMaterializerActivityNode sma = new SplitterMaterializerActivityNode(
- new ActivityId(odId, SPLITTER_MATERIALIZER_ACTIVITY_ID));
+ SplitterMaterializerActivityNode sma =
+ new SplitterMaterializerActivityNode(new ActivityId(odId, SPLITTER_MATERIALIZER_ACTIVITY_ID));
builder.addActivity(this, sma);
builder.addSourceEdge(0, sma, 0);
int pipelineOutputIndex = 0;
int activityId = MATERIALIZE_READER_ACTIVITY_ID;
for (int i = 0; i < outputArity; i++) {
if (outputMaterializationFlags[i]) {
- MaterializeReaderActivityNode mra = new MaterializeReaderActivityNode(
- new ActivityId(odId, activityId++));
+ MaterializeReaderActivityNode mra =
+ new MaterializeReaderActivityNode(new ActivityId(odId, activityId++));
builder.addActivity(this, mra);
builder.addBlockingEdge(sma, mra);
builder.addTargetEdge(i, mra, 0);
@@ -139,6 +139,15 @@
}
@Override
+ public void flush() throws HyracksDataException {
+ if (!requiresMaterialization) {
+ for (IFrameWriter writer : writers) {
+ writer.flush();
+ }
+ }
+ }
+
+ @Override
public void close() throws HyracksDataException {
HyracksDataException hde = null;
try {
@@ -218,4 +227,5 @@
};
}
}
+
}