Refactor General Active Classes
This change is the first of a series of changes aiming to refactor
feed related classes into generic active and feed specific.
In this change, we introduce asterix-active which will be one level
below asterix-external-data. The ultimate goal is to have a very
flexible framewrok for long running activities within asterix.
Change-Id: I0a8f33ee5b45c5e090b08c24a102e369aae43c04
Reviewed-on: https://asterix-gerrit.ics.uci.edu/977
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/asterixdb/asterix-active/pom.xml b/asterixdb/asterix-active/pom.xml
new file mode 100644
index 0000000..140d67a
--- /dev/null
+++ b/asterixdb/asterix-active/pom.xml
@@ -0,0 +1,45 @@
+<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>apache-asterixdb</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-active</artifactId>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-om</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-test-support</artifactId>
+ <version>0.2.18-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-runtime</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-compiler</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
\ No newline at end of file
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
new file mode 100644
index 0000000..e5ccd05
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
@@ -0,0 +1,68 @@
+/*
+ * 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.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;
+
+ public enum EventKind {
+ JOB_START,
+ JOB_FINISH,
+ 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) {
+ this.jobId = jobId;
+ this.eventKind = eventKind;
+ this.entityId = feedId;
+ this.payload = payload;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public EntityId getFeedId() {
+ return entityId;
+ }
+
+ public Serializable getPayload() {
+ return payload;
+ }
+
+ public EventKind getEventKind() {
+ return eventKind;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJob.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJob.java
new file mode 100644
index 0000000..1e3eca1
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJob.java
@@ -0,0 +1,92 @@
+/*
+ * 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.Serializable;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+
+public class ActiveJob implements Serializable {
+ private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(ActiveJob.class.getName());
+ protected final EntityId entityId;
+ protected JobId jobId;
+ protected final Serializable jobObject;
+ protected ActivityState state;
+ protected JobSpecification spec;
+
+ public ActiveJob(EntityId entityId, JobId jobId, ActivityState state, Serializable jobInfo, JobSpecification spec) {
+ this.entityId = entityId;
+ this.state = state;
+ this.jobId = jobId;
+ this.jobObject = jobInfo;
+ this.spec = spec;
+ }
+
+ public ActiveJob(EntityId entityId, ActivityState state, Serializable jobInfo, JobSpecification spec) {
+ this.entityId = entityId;
+ this.state = state;
+ this.jobObject = jobInfo;
+ this.spec = spec;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public void setJobId(JobId jobId) {
+ this.jobId = jobId;
+ }
+
+ public ActivityState getState() {
+ return state;
+ }
+
+ public void setState(ActivityState state) {
+ this.state = state;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(this + " is in " + state + " state.");
+ }
+ }
+
+ public Object getJobObject() {
+ return jobObject;
+ }
+
+ public JobSpecification getSpec() {
+ return spec;
+ }
+
+ public void setSpec(JobSpecification spec) {
+ this.spec = spec;
+ }
+
+ @Override
+ public String toString() {
+ return jobId + " [" + jobObject + "]";
+ }
+
+ public EntityId getEntityId() {
+ return entityId;
+ }
+
+}
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
new file mode 100644
index 0000000..9c69aca
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java
@@ -0,0 +1,194 @@
+/*
+ * 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.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.active.ActiveEvent.EventKind;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+
+public class ActiveJobNotificationHandler implements Runnable {
+ public static final ActiveJobNotificationHandler INSTANCE = new ActiveJobNotificationHandler();
+ public static final String ACTIVE_ENTITY_PROPERTY_NAME = "ActiveJob";
+ private static final Logger LOGGER = Logger.getLogger(ActiveJobNotificationHandler.class.getName());
+ private static final boolean DEBUG = false;
+ private final LinkedBlockingQueue<ActiveEvent> eventInbox;
+ private final Map<EntityId, IActiveEntityEventsListener> entityEventListener;
+ private final Map<JobId, ActiveJob> jobId2ActiveJobInfos;
+
+ private ActiveJobNotificationHandler() {
+ this.eventInbox = new LinkedBlockingQueue<>();
+ this.jobId2ActiveJobInfos = new HashMap<>();
+ this.entityEventListener = new HashMap<>();
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName(ActiveJobNotificationHandler.class.getSimpleName());
+ LOGGER.log(Level.INFO, "Started " + ActiveJobNotificationHandler.class.getSimpleName());
+ while (!Thread.interrupted()) {
+ try {
+ ActiveEvent event = getEventInbox().take();
+ ActiveJob jobInfo = jobId2ActiveJobInfos.get(event.getJobId());
+ 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");
+ }
+ listener.notify(event);
+ if (event.getEventKind() == EventKind.JOB_FINISH) {
+ removeFinishedJob(event.getJobId());
+ removeInactiveListener(listener);
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ } catch (Exception e) {
+ LOGGER.log(Level.SEVERE, "Error handling an active job event", e);
+ }
+ }
+ LOGGER.log(Level.INFO, "Stopped " + ActiveJobNotificationHandler.class.getSimpleName());
+ }
+
+ private void removeFinishedJob(JobId jobId) {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "Removing the job");
+ }
+ jobId2ActiveJobInfos.remove(jobId);
+ }
+
+ private void removeInactiveListener(IActiveEntityEventsListener listener) {
+ if (!listener.isEntityActive()) {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "Removing the listener since it is not active anymore");
+ }
+ entityEventListener.remove(listener.getEntityId());
+ }
+ }
+
+ public IActiveEntityEventsListener getActiveEntityListener(EntityId entityId) {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
+ IActiveEntityEventsListener listener = entityEventListener.get(entityId);
+ LOGGER.log(Level.INFO, "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());
+ }
+ 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);
+ boolean found = jobId2ActiveJobInfos.get(jobId) != null;
+ LOGGER.log(Level.INFO, "Job was found to be: " + (found ? "Active" : "Inactive"));
+ }
+ return jobId2ActiveJobInfos.get(jobId) != null;
+ }
+
+ public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO,
+ "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);
+ }
+ return;
+ } else {
+ monitorJob(jobId, (ActiveJob) property);
+ }
+ if (DEBUG) {
+ boolean found = jobId2ActiveJobInfos.get(jobId) != null;
+ LOGGER.log(Level.INFO, "Job was found to be: " + (found ? "Active" : "Inactive"));
+ }
+ ActiveJob jobInfo = jobId2ActiveJobInfos.get(jobId);
+ if (jobInfo != null) {
+ EntityId entityId = jobInfo.getEntityId();
+ IActiveEntityEventsListener listener = entityEventListener.get(entityId);
+ listener.notifyJobCreation(jobId, jobSpecification);
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "Listener was notified" + jobId);
+ }
+ } else {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "Listener was not notified since it was not registered for the job " + jobId);
+ }
+ }
+ }
+
+ public LinkedBlockingQueue<ActiveEvent> getEventInbox() {
+ return eventInbox;
+ }
+
+ public synchronized IActiveEntityEventsListener[] getEventListeners() {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "getEventListeners() was called");
+ LOGGER.log(Level.INFO, "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());
+ }
+ if (entityEventListener.containsKey(listener.getEntityId())) {
+ throw new HyracksDataException(
+ "Active Entity Listener " + listener.getEntityId() + " is already registered");
+ }
+ entityEventListener.put(listener.getEntityId(), listener);
+ }
+
+ public synchronized void monitorJob(JobId jobId, ActiveJob activeJob) {
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "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"));
+ }
+ if (entityEventListener.containsKey(activeJob.getEntityId())) {
+ if (jobId2ActiveJobInfos.containsKey(jobId)) {
+ LOGGER.severe("Job is already being monitored for job: " + jobId);
+ return;
+ }
+ if (DEBUG) {
+ LOGGER.log(Level.INFO, "monitoring started for job id: " + jobId);
+ }
+ jobId2ActiveJobInfos.put(jobId, activeJob);
+ } else {
+ LOGGER.severe("No listener was found for the entity: " + activeJob.getEntityId());
+ }
+ }
+}
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
new file mode 100644
index 0000000..5992294
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -0,0 +1,70 @@
+/*
+ * 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.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ActiveManager {
+
+ 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;
+ }
+
+ public void registerRuntime(IActiveRuntime runtime) {
+ ActiveRuntimeId id = runtime.getRuntimeId();
+ if (!runtimes.containsKey(id)) {
+ runtimes.put(id, runtime);
+ }
+ }
+
+ public void deregisterRuntime(ActiveRuntimeId id) {
+ runtimes.remove(id);
+ }
+
+ public IActiveRuntime getSubscribableRuntime(ActiveRuntimeId subscribableRuntimeId) {
+ return runtimes.get(subscribableRuntimeId);
+ }
+
+ @Override
+ public String toString() {
+ return ActiveManager.class.getSimpleName() + "[" + nodeId + "]";
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedPartitionStartMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
similarity index 70%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedPartitionStartMessage.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
index 49b23ed..8875647 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedPartitionStartMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivePartitionMessage.java
@@ -16,33 +16,40 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.message;
+package org.apache.asterix.active;
+
+import java.io.Serializable;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.hyracks.api.job.JobId;
-public class FeedPartitionStartMessage extends AbstractApplicationMessage {
+public class ActivePartitionMessage extends AbstractApplicationMessage {
private static final long serialVersionUID = 1L;
- private final FeedId feedId;
+ private final EntityId feedId;
private final JobId jobId;
+ private final Serializable payload;
- public FeedPartitionStartMessage(FeedId feedId, JobId jobId) {
+ public ActivePartitionMessage(EntityId feedId, JobId jobId, Serializable payload) {
this.feedId = feedId;
this.jobId = jobId;
+ this.payload = payload;
}
@Override
public ApplicationMessageType getMessageType() {
- return ApplicationMessageType.FEED_PROVIDER_READY;
+ return ApplicationMessageType.ACTIVE_ENTITY_MESSAGE;
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
public JobId getJobId() {
return jobId;
}
+
+ public Serializable getPayload() {
+ return payload;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
similarity index 77%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
index 8c86d86..8b0914a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntime.java
@@ -16,21 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.runtime;
+package org.apache.asterix.active;
-import org.apache.asterix.external.feed.api.IFeedRuntime;
-
-public class FeedRuntime implements IFeedRuntime {
+public class ActiveRuntime implements IActiveRuntime {
/** A unique identifier for the runtime **/
- protected final FeedRuntimeId runtimeId;
+ protected final ActiveRuntimeId runtimeId;
- public FeedRuntime(FeedRuntimeId runtimeId) {
+ public ActiveRuntime(ActiveRuntimeId runtimeId) {
this.runtimeId = runtimeId;;
}
@Override
- public FeedRuntimeId getRuntimeId() {
+ public ActiveRuntimeId getRuntimeId() {
return runtimeId;
}
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
new file mode 100644
index 0000000..64926fd
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeId.java
@@ -0,0 +1,77 @@
+/*
+ * 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.Serializable;
+
+public class ActiveRuntimeId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final EntityId entityId;
+ private final String runtimeId;
+ private final int partition;
+ private final int hashCode;
+
+ public ActiveRuntimeId(EntityId entityId, String runtimeId, int partition) {
+ this.entityId = entityId;
+ this.runtimeId = runtimeId;
+ this.partition = partition;
+ this.hashCode = toString().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "(" + entityId + ")" + "[" + partition + "]:" + runtimeId;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof ActiveRuntimeId)) {
+ return false;
+ }
+ ActiveRuntimeId other = (ActiveRuntimeId) o;
+ return other.entityId.equals(entityId) && other.getFeedRuntimeType().equals(runtimeId)
+ && other.getPartition() == partition;
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCode;
+ }
+
+ public String getFeedRuntimeType() {
+ return runtimeId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public String getRuntimeType() {
+ return runtimeId;
+ }
+
+ public EntityId getFeedId() {
+ 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
new file mode 100644
index 0000000..9743856
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.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.active;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class ActiveRuntimeManager {
+
+ private static final Logger LOGGER = Logger.getLogger(ActiveRuntimeManager.class.getName());
+ private final Map<ActiveRuntimeId, ActiveRuntime> activeRuntimes;
+
+ private final ExecutorService executorService;
+
+ public ActiveRuntimeManager() {
+ this.activeRuntimes = new ConcurrentHashMap<>();
+ this.executorService = Executors.newCachedThreadPool();
+ }
+
+ public void close() throws IOException {
+ if (executorService != null) {
+ executorService.shutdown();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Shut down executor service for :" + ActiveRuntimeManager.class.getSimpleName());
+ }
+ try {
+ executorService.awaitTermination(10L, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ LOGGER.log(Level.SEVERE, ActiveRuntimeManager.class.getSimpleName()
+ + " was interrupted while waiting for runtime managers to shutdown", e);
+ }
+ if (!executorService.isTerminated()) {
+ LOGGER.severe(ActiveRuntimeManager.class.getSimpleName()
+ + " failed to shutdown successfully. Will be forced to shutdown");
+ executorService.shutdownNow();
+ }
+ }
+ }
+
+ public ActiveRuntime getFeedRuntime(ActiveRuntimeId runtimeId) {
+ return activeRuntimes.get(runtimeId);
+ }
+
+ public void registerRuntime(ActiveRuntimeId runtimeId, ActiveRuntime feedRuntime) {
+ activeRuntimes.put(runtimeId, feedRuntime);
+ }
+
+ public synchronized void deregisterRuntime(ActiveRuntimeId runtimeId) {
+ activeRuntimes.remove(runtimeId);
+ }
+
+ public ExecutorService getExecutorService() {
+ return executorService;
+ }
+
+ public Set<ActiveRuntimeId> getFeedRuntimes() {
+ return activeRuntimes.keySet();
+ }
+
+}
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
new file mode 100644
index 0000000..050426c
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeRegistry.java
@@ -0,0 +1,79 @@
+/*
+ * 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/Activity.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
new file mode 100644
index 0000000..9538118
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+
+public class Activity implements Comparable<Activity> {
+
+ protected int activityId;
+ protected final EntityId activeEntityId;
+ protected final Map<String, String> activityDetails;
+
+ public Activity(EntityId activeEntityId, Map<String, String> activityDetails) {
+ this.activeEntityId = activeEntityId;
+ this.activityDetails = activityDetails;
+ }
+
+ public String getDataverseName() {
+ return activeEntityId.getDataverse();
+ }
+
+ public String getActiveEntityName() {
+ return activeEntityId.getEntityName();
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!(other instanceof Activity)) {
+ return false;
+ }
+ return ((Activity) other).activeEntityId.equals(activeEntityId)
+ && ((Activity) other).getActivityId() != (activityId);
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return activeEntityId + "." + activityId;
+ }
+
+ public int getActivityId() {
+ return activityId;
+ }
+
+ public void setActivityId(int activityId) {
+ this.activityId = activityId;
+ }
+
+ public Map<String, String> getActivityDetails() {
+ return activityDetails;
+ }
+
+ @Override
+ public int compareTo(Activity o) {
+ return o.getActivityId() - this.activityId;
+ }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
similarity index 60%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
copy to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
index 8c86d86..1301535 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
@@ -16,26 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.runtime;
+package org.apache.asterix.active;
-import org.apache.asterix.external.feed.api.IFeedRuntime;
-
-public class FeedRuntime implements IFeedRuntime {
-
- /** A unique identifier for the runtime **/
- protected final FeedRuntimeId runtimeId;
-
- public FeedRuntime(FeedRuntimeId runtimeId) {
- this.runtimeId = runtimeId;;
- }
-
- @Override
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
-
- @Override
- public String toString() {
- return runtimeId.toString();
- }
-}
+// TODO: Document the state machine and its transition.
+public enum ActivityState {
+ CREATED,
+ INACTIVE,
+ ACTIVE,
+ UNDER_RECOVERY,
+ ENDED
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ConcurrentFramePool.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ConcurrentFramePool.java
similarity index 98%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ConcurrentFramePool.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ConcurrentFramePool.java
index e5543d6..afe3b06 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ConcurrentFramePool.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ConcurrentFramePool.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.management;
+package org.apache.asterix.active;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
@@ -26,7 +26,6 @@
import java.util.Map;
import java.util.Map.Entry;
-import org.apache.asterix.external.feed.dataflow.FrameAction;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.log4j.Logger;
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
new file mode 100644
index 0000000..cdf702d
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
@@ -0,0 +1,76 @@
+/*
+ * 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.Serializable;
+
+/**
+ * A unique identifier for a data feed.
+ */
+public class EntityId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String extensionName;
+ private final String dataverse;
+ private final String entityName;
+ private final int hashCode;
+
+ public EntityId(String extentionName, String dataverse, String entityName) {
+ this.extensionName = extentionName;
+ this.dataverse = dataverse;
+ this.entityName = entityName;
+ this.hashCode = toString().hashCode();
+ }
+
+ public String getDataverse() {
+ return dataverse;
+ }
+
+ public String getEntityName() {
+ return entityName;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || !(o instanceof EntityId)) {
+ return false;
+ }
+ if (this == o || ((EntityId) o).getExtensionName().equals(extensionName)
+ && ((EntityId) o).getEntityName().equals(entityName)
+ && ((EntityId) o).getDataverse().equals(dataverse)) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCode;
+ }
+
+ @Override
+ public String toString() {
+ return dataverse + "." + entityName + "(" + extensionName + ")";
+ }
+
+ public String getExtensionName() {
+ return extensionName;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameAction.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/FrameAction.java
similarity index 97%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameAction.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/FrameAction.java
index f02b4aa..849d360 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameAction.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/FrameAction.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.dataflow;
+package org.apache.asterix.active;
import java.nio.ByteBuffer;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
similarity index 60%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
copy to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
index 8c86d86..156576c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
@@ -16,26 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.runtime;
+package org.apache.asterix.active;
-import org.apache.asterix.external.feed.api.IFeedRuntime;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
-public class FeedRuntime implements IFeedRuntime {
+public interface IActiveEntityEventsListener {
- /** A unique identifier for the runtime **/
- protected final FeedRuntimeId runtimeId;
+ public void notify(ActiveEvent message);
- public FeedRuntime(FeedRuntimeId runtimeId) {
- this.runtimeId = runtimeId;;
- }
+ public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification);
- @Override
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
+ public boolean isEntityActive();
- @Override
- public String toString() {
- return runtimeId.toString();
- }
+ public EntityId getEntityId();
+
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveMessage.java
similarity index 79%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveMessage.java
index 15e2de6..e4c7171 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveMessage.java
@@ -16,16 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.active;
import java.io.Serializable;
import org.apache.hyracks.api.dataflow.value.JSONSerializable;
/**
- * A control message exchanged between {@Link IFeedManager} and {@Link CentralFeedManager} that requests for an action or reporting of an event
+ * @deprecated
+ * This interface is expected to go away. instead, one should use the IMessageBroker interfaces to exchange
+ * messages
*/
-public interface IFeedMessage extends Serializable, JSONSerializable {
+@Deprecated
+public interface IActiveMessage extends Serializable, JSONSerializable {
public enum MessageType {
END
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
similarity index 60%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
copy to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
index 8c86d86..32c5c50 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
@@ -16,26 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.runtime;
+package org.apache.asterix.active;
-import org.apache.asterix.external.feed.api.IFeedRuntime;
+@FunctionalInterface
+public interface IActiveRuntime {
-public class FeedRuntime implements IFeedRuntime {
-
- /** A unique identifier for the runtime **/
- protected final FeedRuntimeId runtimeId;
-
- public FeedRuntime(FeedRuntimeId runtimeId) {
- this.runtimeId = runtimeId;;
- }
-
- @Override
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
-
- @Override
- public String toString() {
- return runtimeId.toString();
- }
+ /**
+ * @return the unique runtime id associated with the feedRuntime
+ */
+ public ActiveRuntimeId getRuntimeId();
}
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
new file mode 100644
index 0000000..b2c6f8e
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntimeRegistry.java
@@ -0,0 +1,49 @@
+/*
+ * 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-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 327154f..d623830 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
@@ -22,10 +22,10 @@
import java.util.List;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.watch.FeedActivity.FeedActivityDetails;
import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.external.util.FeedUtils;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.asterix.metadata.declared.AqlDataSource;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.metadata.declared.AqlSourceId;
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 a161717..83e4375 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
@@ -47,7 +47,7 @@
*/
public abstract class AbstractLangTranslator {
- protected static final Logger LOGGER = Logger.getLogger(AbstractLangTranslator.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(AbstractLangTranslator.class.getName());
protected static final Map<String, BuiltinType> builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
@@ -107,7 +107,7 @@
String message = null;
String dataverse = defaultDataverse != null ? defaultDataverse.getDataverseName() : null;
switch (stmt.getKind()) {
- case INSERT:
+ case Statement.INSERT:
InsertStatement insertStmt = (InsertStatement) stmt;
if (insertStmt.getDataverseName() != null) {
dataverse = insertStmt.getDataverseName().getValue();
@@ -119,7 +119,7 @@
}
break;
- case DELETE:
+ case Statement.DELETE:
DeleteStatement deleteStmt = (DeleteStatement) stmt;
if (deleteStmt.getDataverseName() != null) {
dataverse = deleteStmt.getDataverseName().getValue();
@@ -131,7 +131,7 @@
}
break;
- case NODEGROUP_DROP:
+ case Statement.NODEGROUP_DROP:
String nodegroupName = ((NodeGroupDropStatement) stmt).getNodeGroupName().getValue();
invalidOperation = MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME.equals(nodegroupName);
if (invalidOperation) {
@@ -139,16 +139,16 @@
}
break;
- case DATAVERSE_DROP:
+ case Statement.DATAVERSE_DROP:
DataverseDropStatement dvDropStmt = (DataverseDropStatement) stmt;
- invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME
- .equals(dvDropStmt.getDataverseName().getValue());
+ invalidOperation =
+ MetadataConstants.METADATA_DATAVERSE_NAME.equals(dvDropStmt.getDataverseName().getValue());
if (invalidOperation) {
message = "Cannot drop dataverse:" + dvDropStmt.getDataverseName().getValue();
}
break;
- case DATASET_DROP:
+ case Statement.DATASET_DROP:
DropStatement dropStmt = (DropStatement) stmt;
if (dropStmt.getDataverseName() != null) {
dataverse = dropStmt.getDataverseName().getValue();
@@ -159,7 +159,7 @@
+ MetadataConstants.METADATA_DATAVERSE_NAME;
}
break;
- case DATASET_DECL:
+ case Statement.DATASET_DECL:
DatasetDecl datasetStmt = (DatasetDecl) stmt;
Map<String, String> hints = datasetStmt.getHints();
if (hints != null && !hints.isEmpty()) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 9d092a7..7a3b2a4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -24,7 +24,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
import org.apache.asterix.lang.common.base.Expression;
-import org.apache.asterix.lang.common.base.Statement.Kind;
+import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.om.types.IAType;
@@ -38,7 +38,7 @@
public static interface ICompiledStatement {
- public Kind getKind();
+ public byte getKind();
}
public static class CompiledDatasetDropStatement implements ICompiledStatement {
@@ -59,8 +59,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DATASET_DROP;
+ public byte getKind() {
+ return Statement.DATASET_DROP;
}
}
@@ -83,8 +83,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_DATAVERSE;
+ public byte getKind() {
+ return Statement.CREATE_DATAVERSE;
}
}
@@ -100,8 +100,8 @@
}
@Override
- public Kind getKind() {
- return Kind.NODEGROUP_DROP;
+ public byte getKind() {
+ return Statement.NODEGROUP_DROP;
}
}
@@ -129,8 +129,8 @@
}
@Override
- public Kind getKind() {
- return Kind.INDEX_DROP;
+ public byte getKind() {
+ return Statement.INDEX_DROP;
}
}
@@ -152,8 +152,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DATAVERSE_DROP;
+ public byte getKind() {
+ return Statement.DATAVERSE_DROP;
}
}
@@ -169,8 +169,8 @@
}
@Override
- public Kind getKind() {
- return Kind.TYPE_DROP;
+ public byte getKind() {
+ return Statement.TYPE_DROP;
}
}
@@ -241,8 +241,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_INDEX;
+ public byte getKind() {
+ return Statement.CREATE_INDEX;
}
}
@@ -285,8 +285,8 @@
}
@Override
- public Kind getKind() {
- return Kind.LOAD;
+ public byte getKind() {
+ return Statement.LOAD;
}
}
@@ -322,8 +322,8 @@
}
@Override
- public Kind getKind() {
- return Kind.INSERT;
+ public byte getKind() {
+ return Statement.INSERT;
}
}
@@ -334,8 +334,8 @@
}
@Override
- public Kind getKind() {
- return Kind.UPSERT;
+ public byte getKind() {
+ return Statement.UPSERT;
}
}
@@ -380,8 +380,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CONNECT_FEED;
+ public byte getKind() {
+ return Statement.CONNECT_FEED;
}
public String getPolicyName() {
@@ -405,7 +405,7 @@
}
public String getFeedName() {
- return request.getReceivingFeedId().getFeedName();
+ return request.getReceivingFeedId().getEntityName();
}
@Override
@@ -418,8 +418,8 @@
}
@Override
- public Kind getKind() {
- return Kind.SUBSCRIBE_FEED;
+ public byte getKind() {
+ return Statement.SUBSCRIBE_FEED;
}
}
@@ -449,8 +449,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DISCONNECT_FEED;
+ public byte getKind() {
+ return Statement.DISCONNECT_FEED;
}
}
@@ -494,8 +494,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DELETE;
+ public byte getKind() {
+ return Statement.DELETE;
}
}
@@ -518,8 +518,8 @@
}
@Override
- public Kind getKind() {
- return Kind.COMPACT;
+ public byte getKind() {
+ return Statement.COMPACT;
}
}
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 b8786d1..3689357 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
@@ -39,6 +39,7 @@
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.Statement;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -247,8 +248,8 @@
}
List<String> additionalFilteringField = DatasetUtils.getFilterField(targetDatasource.getDataset());
- List<LogicalVariable> additionalFilteringVars = null;
- List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions = null;
+ List<LogicalVariable> additionalFilteringVars;
+ List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions;
List<Mutable<ILogicalExpression>> additionalFilteringExpressions = null;
AssignOperator additionalFilteringAssign = null;
if (additionalFilteringField != null) {
@@ -280,7 +281,7 @@
@SuppressWarnings("unchecked")
@Override
public ILogicalPlan translate(Query expr, String outputDatasetName, ICompiledDmlStatement stmt)
- throws AlgebricksException, AsterixException {
+ throws AlgebricksException {
Pair<ILogicalOperator, LogicalVariable> p =
expr.accept(this, new MutableObject<>(new EmptyTupleSourceOperator()));
ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<>();
@@ -348,8 +349,8 @@
AssignOperator assign = new AssignOperator(vars, exprs);
List<String> additionalFilteringField = DatasetUtils.getFilterField(targetDatasource.getDataset());
- List<LogicalVariable> additionalFilteringVars = null;
- List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions = null;
+ List<LogicalVariable> additionalFilteringVars;
+ List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions;
List<Mutable<ILogicalExpression>> additionalFilteringExpressions = null;
AssignOperator additionalFilteringAssign = null;
if (additionalFilteringField != null) {
@@ -369,148 +370,31 @@
}
Mutable<ILogicalExpression> varRef = new MutableObject<>(new VariableReferenceExpression(resVar));
- ILogicalOperator leafOperator = null;
-
+ ILogicalOperator leafOperator;
switch (stmt.getKind()) {
- case INSERT: {
- if (targetDatasource.getDataset().hasMetaPart()) {
- throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
- + ": insert into dataset is not supported on Datasets with Meta records");
- }
- InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
- insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
- insertOp.getInputs().add(new MutableObject<>(assign));
- leafOperator = new SinkOperator();
- leafOperator.getInputs().add(new MutableObject<>(insertOp));
+ case Statement.INSERT:
+ leafOperator = translateInsert(targetDatasource, varRef, varRefsForLoading,
+ additionalFilteringExpressions, assign);
break;
- }
- case UPSERT: {
- if (targetDatasource.getDataset().hasMetaPart()) {
- 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)));
- }
- leafOperator = new SinkOperator();
- leafOperator.getInputs().add(new MutableObject<>(upsertOp));
+ case Statement.UPSERT:
+ leafOperator = translateUpsert(targetDatasource, varRef, varRefsForLoading,
+ additionalFilteringExpressions, assign, additionalFilteringField);
break;
- }
- case DELETE: {
- if (targetDatasource.getDataset().hasMetaPart()) {
- throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
- + ": delete from dataset is not supported on Datasets with Meta records");
- }
- InsertDeleteUpsertOperator deleteOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteUpsertOperator.Kind.DELETE, false);
- deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
- deleteOp.getInputs().add(new MutableObject<>(assign));
- leafOperator = new SinkOperator();
- leafOperator.getInputs().add(new MutableObject<>(deleteOp));
+ case Statement.DELETE:
+ leafOperator = translateDelete(targetDatasource, varRef, varRefsForLoading,
+ additionalFilteringExpressions, assign);
break;
- }
- case CONNECT_FEED: {
- InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
- insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
- insertOp.getInputs().add(new MutableObject<>(assign));
- leafOperator = new SinkOperator();
- leafOperator.getInputs().add(new MutableObject<>(insertOp));
+ case Statement.CONNECT_FEED:
+ leafOperator = translateConnectFeed(targetDatasource, varRef, varRefsForLoading,
+ additionalFilteringExpressions, assign);
break;
- }
- case SUBSCRIBE_FEED: {
- // if the feed is a change feed (i.e, performs different operations), we need to project op variable
- CompiledSubscribeFeedStatement sfs = (CompiledSubscribeFeedStatement) stmt;
- InsertDeleteUpsertOperator feedModificationOp;
- AssignOperator metaAndKeysAssign = null;
- List<LogicalVariable> metaAndKeysVars = null;
- List<Mutable<ILogicalExpression>> metaAndKeysExprs = null;
- List<Mutable<ILogicalExpression>> metaExpSingletonList = null;
- boolean isChangeFeed =
- FeedMetadataUtil.isChangeFeed(metadataProvider, sfs.getDataverseName(), sfs.getFeedName());
- if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
- metaAndKeysVars = new ArrayList<>();
- metaAndKeysExprs = new ArrayList<>();
- }
- if (targetDatasource.getDataset().hasMetaPart()) {
- // 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);
- }
- if (isChangeFeed) {
- 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));
- }
- } else {
- feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
- metaExpSingletonList, InsertDeleteUpsertOperator.Kind.INSERT, false);
- feedModificationOp.getInputs().add(new MutableObject<>(assign));
- }
- if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
- metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
- metaAndKeysAssign.getInputs().add(project.getInputs().get(0));
- project.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
- }
- feedModificationOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
- leafOperator = new SinkOperator();
- leafOperator.getInputs().add(new MutableObject<>(feedModificationOp));
+ case Statement.SUBSCRIBE_FEED:
+ leafOperator = translateSubscribeFeed((CompiledSubscribeFeedStatement) stmt, targetDatasource,
+ unnestVar, project, exprs, resVar, varRefsForLoading, varRef, assign,
+ additionalFilteringField, additionalFilteringAssign, additionalFilteringExpressions);
break;
- }
default:
- break;
+ throw new AlgebricksException("Unsupported statement kind " + stmt.getKind());
}
topOp = leafOperator;
}
@@ -520,6 +404,162 @@
return plan;
}
+ private ILogicalOperator translateConnectFeed(DatasetDataSource targetDatasource,
+ Mutable<ILogicalExpression> varRef, List<Mutable<ILogicalExpression>> varRefsForLoading,
+ List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign) {
+ InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
+ varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
+ insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ insertOp.getInputs().add(new MutableObject<>(assign));
+ SinkOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<>(insertOp));
+ return leafOperator;
+ }
+
+ private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
+ List<Mutable<ILogicalExpression>> varRefsForLoading,
+ List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign)
+ throws AlgebricksException {
+ if (targetDatasource.getDataset().hasMetaPart()) {
+ throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
+ + ": delete from dataset is not supported on Datasets with Meta records");
+ }
+ InsertDeleteUpsertOperator deleteOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
+ varRefsForLoading, InsertDeleteUpsertOperator.Kind.DELETE, false);
+ deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ deleteOp.getInputs().add(new MutableObject<>(assign));
+ SinkOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<>(deleteOp));
+ return leafOperator;
+ }
+
+ private ILogicalOperator translateSubscribeFeed(CompiledSubscribeFeedStatement sfs,
+ DatasetDataSource targetDatasource, LogicalVariable unnestVar, ProjectOperator project,
+ ArrayList<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar,
+ List<Mutable<ILogicalExpression>> varRefsForLoading, Mutable<ILogicalExpression> varRef,
+ ILogicalOperator assign, List<String> additionalFilteringField, AssignOperator additionalFilteringAssign,
+ List<Mutable<ILogicalExpression>> additionalFilteringExpressions) throws AlgebricksException {
+ // if the feed is a change feed (i.e, performs different operations), we need to project op variable
+ InsertDeleteUpsertOperator feedModificationOp;
+ AssignOperator metaAndKeysAssign;
+ List<LogicalVariable> metaAndKeysVars = null;
+ List<Mutable<ILogicalExpression>> metaAndKeysExprs = null;
+ List<Mutable<ILogicalExpression>> metaExpSingletonList = null;
+ boolean isChangeFeed =
+ FeedMetadataUtil.isChangeFeed(metadataProvider, sfs.getDataverseName(), sfs.getFeedName());
+ if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
+ metaAndKeysVars = new ArrayList<>();
+ metaAndKeysExprs = new ArrayList<>();
+ if (targetDatasource.getDataset().hasMetaPart()) {
+ // 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);
+ }
+ }
+ if (isChangeFeed) {
+ 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));
+ }
+ } else {
+ feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
+ metaExpSingletonList, InsertDeleteUpsertOperator.Kind.INSERT, false);
+ feedModificationOp.getInputs().add(new MutableObject<>(assign));
+ }
+ if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
+ 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;
+ }
+
+ 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()) {
+ 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)));
+ }
+ SinkOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<>(upsertOp));
+ return leafOperator;
+ }
+
+ private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
+ List<Mutable<ILogicalExpression>> varRefsForLoading,
+ List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign)
+ throws AlgebricksException {
+ if (targetDatasource.getDataset().hasMetaPart()) {
+ throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
+ + ": insert into dataset is not supported on Datasets with Meta records");
+ }
+ InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
+ varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
+ insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ insertOp.getInputs().add(new MutableObject<>(assign));
+ SinkOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<>(insertOp));
+ return leafOperator;
+ }
+
private DatasetDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
String datasetName) throws AlgebricksException {
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
@@ -534,9 +574,8 @@
IAType metaItemType =
metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
INodeDomain domain = metadataProvider.findNodeDomain(dataset.getNodeGroupName());
- DatasetDataSource dataSource = new DatasetDataSource(sourceId, dataset, itemType, metaItemType,
- AqlDataSourceType.INTERNAL_DATASET, dataset.getDatasetDetails(), domain);
- return dataSource;
+ return new DatasetDataSource(sourceId, dataset, itemType, metaItemType, AqlDataSourceType.INTERNAL_DATASET,
+ dataset.getDatasetDetails(), domain);
}
private FileSplit getDefaultOutputFileLocation() throws MetadataException {
@@ -552,23 +591,17 @@
throws AsterixException {
LogicalVariable v;
ILogicalOperator returnedOp;
-
- switch (lc.getBindingExpr().getKind()) {
- case VARIABLE_EXPRESSION: {
- v = context.newVar(lc.getVarExpr());
- LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
- returnedOp = new AssignOperator(v, new MutableObject<>(new VariableReferenceExpression(prev)));
- returnedOp.getInputs().add(tupSource);
- break;
- }
- default: {
- v = context.newVar(lc.getVarExpr());
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo =
- langExprToAlgExpression(lc.getBindingExpr(), tupSource);
- returnedOp = new AssignOperator(v, new MutableObject<>(eo.first));
- returnedOp.getInputs().add(eo.second);
- break;
- }
+ if (lc.getBindingExpr().getKind() == Kind.VARIABLE_EXPRESSION) {
+ v = context.newVar(lc.getVarExpr());
+ LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
+ returnedOp = new AssignOperator(v, new MutableObject<>(new VariableReferenceExpression(prev)));
+ returnedOp.getInputs().add(tupSource);
+ } else {
+ v = context.newVar(lc.getVarExpr());
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo =
+ langExprToAlgExpression(lc.getBindingExpr(), tupSource);
+ returnedOp = new AssignOperator(v, new MutableObject<>(eo.first));
+ returnedOp.getInputs().add(eo.second);
}
return new Pair<>(returnedOp, v);
}
@@ -621,18 +654,16 @@
for (Expression expr : fcall.getExprList()) {
switch (expr.getKind()) {
- case VARIABLE_EXPRESSION: {
+ case VARIABLE_EXPRESSION:
LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
args.add(new MutableObject<>(new VariableReferenceExpression(var)));
break;
- }
- case LITERAL_EXPRESSION: {
+ case LITERAL_EXPRESSION:
LiteralExpr val = (LiteralExpr) expr;
args.add(new MutableObject<>(new ConstantExpression(
new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue())))));
break;
- }
- default: {
+ default:
Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, topOp);
AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
args.add(new MutableObject<>(eo.first));
@@ -640,7 +671,6 @@
topOp = eo.second;
}
break;
- }
}
}
@@ -677,7 +707,7 @@
if (function == null) {
return null;
}
- AbstractFunctionCallExpression f = null;
+ AbstractFunctionCallExpression f;
if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
IFunctionInfo finfo = ExternalFunctionCompilerUtil
.getExternalFunctionInfo(metadataProvider.getMetadataTxnContext(), function);
@@ -694,7 +724,7 @@
private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
List<Mutable<ILogicalExpression>> args) {
- AbstractFunctionCallExpression f = null;
+ AbstractFunctionCallExpression f;
FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionName, arity);
AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
@@ -1243,17 +1273,15 @@
protected Pair<ILogicalExpression, Mutable<ILogicalOperator>> langExprToAlgExpression(Expression expr,
Mutable<ILogicalOperator> topOpRef) throws AsterixException {
switch (expr.getKind()) {
- case VARIABLE_EXPRESSION: {
+ case VARIABLE_EXPRESSION:
VariableReferenceExpression ve =
new VariableReferenceExpression(context.getVar(((VariableExpr) expr).getVar().getId()));
return new Pair<>(ve, topOpRef);
- }
- case LITERAL_EXPRESSION: {
+ case LITERAL_EXPRESSION:
LiteralExpr val = (LiteralExpr) expr;
return new Pair<>(new ConstantExpression(
new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOpRef);
- }
- default: {
+ default:
if (expressionNeedsNoNesting(expr)) {
Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOpRef);
ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
@@ -1279,7 +1307,6 @@
return new Pair<>(new VariableReferenceExpression(p.second), new MutableObject<>(s));
}
}
- }
}
}
@@ -1320,11 +1347,9 @@
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(exprs.get(i), topOp);
topOp = p.second;
// now look at the operator
- if (i < nOps) {
- if (ops.get(i) != opLogical) {
- throw new TranslationException(
- "Unexpected operator " + ops.get(i) + " in an OperatorExpr starting with " + opLogical);
- }
+ if (i < nOps && ops.get(i) != opLogical) {
+ throw new TranslationException(
+ "Unexpected operator " + ops.get(i) + " in an OperatorExpr starting with " + opLogical);
}
f.getArguments().add(new MutableObject<>(p.first));
}
@@ -1346,7 +1371,7 @@
}
protected <T> List<T> mkSingletonArrayList(T item) {
- ArrayList<T> array = new ArrayList<T>(1);
+ ArrayList<T> array = new ArrayList<>(1);
array.add(item);
return array;
}
@@ -1356,22 +1381,16 @@
argRefs.add(new MutableObject<>(expr));
switch (expr.getExpressionTag()) {
case CONSTANT:
- case VARIABLE: {
+ case VARIABLE:
return new UnnestingFunctionCallExpression(
FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION), argRefs);
- }
- case FUNCTION_CALL: {
+ case FUNCTION_CALL:
AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
- if (fce.getKind() == FunctionKind.UNNEST) {
- return expr;
- } else {
- return new UnnestingFunctionCallExpression(
- FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION), argRefs);
- }
- }
- default: {
+ return (fce.getKind() == FunctionKind.UNNEST) ? expr
+ : new UnnestingFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION), argRefs);
+ default:
return expr;
- }
}
}
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 6fb82ef..bef8a3a 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
@@ -41,7 +41,7 @@
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
import org.apache.asterix.lang.common.base.IQueryRewriter;
import org.apache.asterix.lang.common.base.IRewriterFactory;
-import org.apache.asterix.lang.common.base.Statement.Kind;
+import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
@@ -204,12 +204,12 @@
org.apache.asterix.common.transactions.JobId asterixJobId = JobIdFactory.generateJobId();
queryMetadataProvider.setJobId(asterixJobId);
- ILangExpressionToPlanTranslator t = translatorFactory.createExpressionToPlanTranslator(queryMetadataProvider,
- varCounter);
+ ILangExpressionToPlanTranslator t =
+ translatorFactory.createExpressionToPlanTranslator(queryMetadataProvider, varCounter);
ILogicalPlan plan;
// statement = null when it's a query
- if (statement == null || statement.getKind() != Kind.LOAD) {
+ if (statement == null || statement.getKind() != Statement.LOAD) {
plan = t.translate(rwQ, outputDatasetName, statement);
} else {
plan = t.translateLoad(statement);
@@ -219,7 +219,7 @@
conf.out().println();
printPlanPrefix(conf, "Logical plan");
- if (rwQ != null || (statement != null && statement.getKind() == Kind.LOAD)) {
+ if (rwQ != null || (statement != null && statement.getKind() == Statement.LOAD)) {
LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor(conf.out());
PlanPrettyPrinter.printPlan(plan, pvisitor, 0);
}
@@ -243,8 +243,8 @@
OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalGroupBy(groupFrameLimit);
OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesForJoin(joinFrameLimit);
- HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder(
- AqlOptimizationContextFactory.INSTANCE);
+ HeuristicCompilerFactoryBuilder builder =
+ new HeuristicCompilerFactoryBuilder(AqlOptimizationContextFactory.INSTANCE);
builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
builder.setLogicalRewrites(buildDefaultLogicalRewrites());
builder.setPhysicalRewrites(buildDefaultPhysicalRewrites());
@@ -271,7 +271,7 @@
PlanPrettyPrinter.printPhysicalOps(plan, buffer, 0);
} else {
printPlanPrefix(conf, "Optimized logical plan");
- if (rwQ != null || (statement != null && statement.getKind() == Kind.LOAD)) {
+ if (rwQ != null || (statement != null && statement.getKind() == Statement.LOAD)) {
LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor(conf.out());
PlanPrettyPrinter.printPlan(plan, pvisitor, 0);
}
@@ -315,8 +315,8 @@
builder.setTypeTraitProvider(format.getTypeTraitProvider());
builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
- JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(asterixJobId,
- queryMetadataProvider.isWriteTransaction());
+ JobEventListenerFactory jobEventListenerFactory =
+ new JobEventListenerFactory(asterixJobId, queryMetadataProvider.isWriteTransaction());
JobSpecification spec = compiler.createJob(AsterixAppContextInfo.getInstance(), jobEventListenerFactory);
if (conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
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/api/common/AsterixAppRuntimeContext.java
index 1545325..4bb09d4 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/api/common/AsterixAppRuntimeContext.java
@@ -25,6 +25,7 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.active.ActiveManager;
import org.apache.asterix.common.api.AsterixThreadExecutor;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
@@ -52,7 +53,6 @@
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.library.ExternalLibraryManager;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataNode;
@@ -121,7 +121,7 @@
private IIOManager ioManager;
private boolean isShuttingdown;
- private FeedManager feedManager;
+ private ActiveManager activeManager;
private IReplicationChannel replicationChannel;
private IReplicationManager replicationManager;
@@ -151,13 +151,13 @@
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;
}
@Override
- public void initialize(boolean initialRun) throws IOException, ACIDException, AsterixException {
+ public void initialize(boolean initialRun) throws IOException, ACIDException {
Logger.getLogger("org.apache").setLevel(externalProperties.getLogLevel());
threadExecutor = new AsterixThreadExecutor(ncApplicationContext.getThreadFactory());
@@ -173,14 +173,15 @@
metadataMergePolicyFactory = new PrefixMergePolicyFactory();
- ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
- ioManager, ncApplicationContext.getNodeId(), metadataProperties);
+ ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
+ new PersistentLocalResourceRepositoryFactory(ioManager, ncApplicationContext.getNodeId(),
+ metadataProperties);
- localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
- .createRepository();
+ localResourceRepository =
+ (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
- IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
- this);
+ IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider =
+ new AsterixAppRuntimeContextProviderForRecovery(this);
txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider,
txnProperties);
@@ -198,8 +199,8 @@
isShuttingdown = false;
- feedManager = new FeedManager(ncApplicationContext.getNodeId(), feedProperties,
- compilerProperties.getFrameSize());
+ activeManager = new ActiveManager(ncApplicationContext.getNodeId(),
+ feedProperties.getMemoryComponentGlobalBudget(), compilerProperties.getFrameSize());
if (replicationProperties.isReplicationEnabled()) {
String nodeId = ncApplicationContext.getNodeId();
@@ -238,9 +239,9 @@
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());
@@ -386,8 +387,8 @@
}
@Override
- public FeedManager getFeedManager() {
- return feedManager;
+ public ActiveManager getFeedManager() {
+ return activeManager;
}
@Override
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 f9dc45c..e799dd0 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,39 +18,30 @@
*/
package org.apache.asterix.api.http.servlet;
-import java.util.ArrayList;
import java.util.List;
import javax.servlet.http.HttpServletRequest;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement.Kind;
+import org.apache.asterix.lang.common.base.Statement;
public class AQLAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
-
private static final String AQL_STMT_PARAM_NAME = "aql";
-
- private static final List<Kind> allowedStatements = new ArrayList<>();
+ private static final List<Byte> allowedStatements = Statement.VALUES;
public AQLAPIServlet(ILangCompilationProvider compilationProvider) {
super(compilationProvider);
}
- static {
- for (Kind k : Kind.values()) {
- allowedStatements.add(k);
- }
- }
-
@Override
protected String getQueryParameter(HttpServletRequest request) {
return request.getParameter(AQL_STMT_PARAM_NAME);
}
@Override
- protected List<Kind> getAllowedStatements() {
+ protected List<Byte> getAllowedStatements() {
return allowedStatements;
}
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 0254d6d..ec47276 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
@@ -25,7 +25,6 @@
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.base.Statement.Kind;
public class DDLAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
@@ -34,19 +33,23 @@
super(compilationProvider);
}
+ @Override
protected String getQueryParameter(HttpServletRequest request) {
return request.getParameter("ddl");
}
- protected List<Statement.Kind> getAllowedStatements() {
- Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DATAVERSE_DROP, Kind.DATASET_DECL, Kind.NODEGROUP_DECL,
- Kind.NODEGROUP_DROP, Kind.TYPE_DECL, Kind.TYPE_DROP, Kind.CREATE_INDEX, Kind.INDEX_DECL,
- Kind.CREATE_DATAVERSE, Kind.DATASET_DROP, Kind.INDEX_DROP, Kind.CREATE_FUNCTION, Kind.FUNCTION_DROP,
- Kind.CREATE_PRIMARY_FEED, Kind.CREATE_SECONDARY_FEED, Kind.DROP_FEED, Kind.CREATE_FEED_POLICY,
- Kind.DROP_FEED_POLICY };
+ @Override
+ protected List<Byte> getAllowedStatements() {
+ Byte[] statementsArray = { Statement.DATAVERSE_DECL, Statement.DATAVERSE_DROP, Statement.DATASET_DECL,
+ Statement.NODEGROUP_DECL, Statement.NODEGROUP_DROP, Statement.TYPE_DECL, Statement.TYPE_DROP,
+ Statement.CREATE_INDEX, Statement.INDEX_DECL, Statement.CREATE_DATAVERSE, Statement.DATASET_DROP,
+ Statement.INDEX_DROP, Statement.CREATE_FUNCTION, Statement.FUNCTION_DROP, Statement.CREATE_PRIMARY_FEED,
+ Statement.CREATE_SECONDARY_FEED, Statement.DROP_FEED, Statement.CREATE_FEED_POLICY,
+ Statement.DROP_FEED_POLICY };
return Arrays.asList(statementsArray);
}
+ @Override
protected String getErrorMessage() {
return "Invalid statement: Non-DDL statement %s to the DDL API.";
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
index 8bc613a..78c68e1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
@@ -32,13 +32,14 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.watch.FeedActivity;
import org.apache.asterix.external.feed.watch.FeedActivity.FeedActivityDetails;
public class FeedServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
+ private static final String FEED_EXTENSION_NAME = "Feed";
@Override
public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
@@ -100,7 +101,8 @@
String store = activity.getFeedActivityDetails().get(FeedActivityDetails.STORAGE_LOCATIONS);
FeedConnectionId connectionId = new FeedConnectionId(
- new FeedId(activity.getDataverseName(), activity.getFeedName()), activity.getDatasetName());
+ new EntityId(FEED_EXTENSION_NAME, activity.getDataverseName(), activity.getFeedName()),
+ activity.getDatasetName());
int intakeRate = 0;
int storeRate = 0;
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 9ada5ff..040ac1a 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
@@ -25,7 +25,6 @@
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.base.Statement.Kind;
public class QueryAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
@@ -34,16 +33,19 @@
super(compilationProvider);
}
+ @Override
protected String getQueryParameter(HttpServletRequest request) {
return request.getParameter("query");
}
- protected List<Statement.Kind> getAllowedStatements() {
- Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.FUNCTION_DECL, Kind.QUERY, Kind.SET, Kind.WRITE,
- Kind.RUN };
+ @Override
+ protected List<Byte> getAllowedStatements() {
+ Byte[] statementsArray = { Statement.DATAVERSE_DECL, Statement.FUNCTION_DECL, Statement.QUERY, Statement.SET,
+ Statement.WRITE, Statement.RUN };
return Arrays.asList(statementsArray);
}
+ @Override
protected String getErrorMessage() {
return "Invalid statement: Non-query statement %s to the query API.";
}
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 95e55f6..7ca1442 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
@@ -40,7 +40,6 @@
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.lang.common.base.Statement.Kind;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.result.ResultReader;
import org.apache.asterix.result.ResultUtils;
@@ -197,8 +196,8 @@
} 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);
- JSONObject errorResp = ResultUtils.getErrorResponse(2, errorMessage, "",
- ResultUtils.extractFullStackTrace(pe));
+ JSONObject errorResp =
+ ResultUtils.getErrorResponse(2, errorMessage, "", ResultUtils.extractFullStackTrace(pe));
sessionConfig.out().write(errorResp.toString());
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
} catch (Exception e) {
@@ -231,7 +230,7 @@
protected abstract String getQueryParameter(HttpServletRequest request);
- protected abstract List<Kind> getAllowedStatements();
+ protected abstract List<Byte> getAllowedStatements();
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 5c76c40..9f762f8 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
@@ -25,7 +25,6 @@
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.base.Statement.Kind;
public class UpdateAPIServlet extends RESTAPIServlet {
private static final long serialVersionUID = 1L;
@@ -40,10 +39,11 @@
}
@Override
- protected List<Statement.Kind> getAllowedStatements() {
- Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DELETE, Kind.INSERT, Kind.UPSERT, Kind.UPDATE,
- Kind.DML_CMD_LIST, Kind.LOAD, Kind.CONNECT_FEED, Kind.DISCONNECT_FEED, Kind.SET, Kind.COMPACT,
- Kind.EXTERNAL_DATASET_REFRESH, Kind.RUN };
+ protected List<Byte> getAllowedStatements() {
+ Byte[] statementsArray =
+ { Statement.DATAVERSE_DECL, Statement.DELETE, Statement.INSERT, Statement.UPSERT, Statement.UPDATE,
+ Statement.DML_CMD_LIST, Statement.LOAD, Statement.CONNECT_FEED, Statement.DISCONNECT_FEED,
+ Statement.SET, Statement.COMPACT, Statement.EXTERNAL_DATASET_REFRESH, Statement.RUN };
return Arrays.asList(statementsArray);
}
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
new file mode 100644
index 0000000..c44ffc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ActiveLifecycleListener.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.external;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.ActiveJobNotificationHandler;
+import org.apache.asterix.active.ActivePartitionMessage;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
+import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.job.JobId;
+
+public class ActiveLifecycleListener implements IJobLifecycleListener {
+
+ private static final Logger LOGGER = Logger.getLogger(ActiveLifecycleListener.class.getName());
+ public static final ActiveLifecycleListener INSTANCE = new ActiveLifecycleListener();
+
+ private final LinkedBlockingQueue<ActiveEvent> jobEventInbox;
+ private final ExecutorService executorService;
+
+ private ActiveLifecycleListener() {
+ jobEventInbox = ActiveJobNotificationHandler.INSTANCE.getEventInbox();
+ executorService = Executors.newSingleThreadExecutor();
+ executorService.execute(ActiveJobNotificationHandler.INSTANCE);
+ }
+
+ @Override
+ public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
+ if (ActiveJobNotificationHandler.INSTANCE.isActiveJob(jobId)) {
+ jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_START));
+ }
+ }
+
+ @Override
+ public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
+ if (ActiveJobNotificationHandler.INSTANCE.isActiveJob(jobId)) {
+ jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_FINISH));
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
+ }
+ }
+ }
+
+ @Override
+ public void notifyJobCreation(JobId jobId, IActivityClusterGraphGeneratorFactory acggf) throws HyracksException {
+ ActiveJobNotificationHandler.INSTANCE.notifyJobCreation(jobId, acggf.getJobSpecification());
+ }
+
+ 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()));
+ }
+ }
+
+ public void stop() {
+ executorService.shutdown();
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java
deleted file mode 100644
index cfc2125..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java
+++ /dev/null
@@ -1,679 +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.app.external;
-
-import java.rmi.RemoteException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.app.external.FeedLifecycleListener.FeedEvent;
-import org.apache.asterix.app.external.FeedWorkCollection.SubscribeFeedWork;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.external.feed.api.FeedOperationCounter;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.api.IFeedJoint.State;
-import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber;
-import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber.FeedLifecycleEvent;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.asterix.external.feed.management.FeedWorkManager;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
-import org.apache.asterix.external.feed.watch.FeedJobInfo;
-import org.apache.asterix.external.feed.watch.FeedJobInfo.FeedJobState;
-import org.apache.asterix.external.feed.watch.FeedJobInfo.JobType;
-import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobInfo;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.api.job.JobStatus;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
-
-public class FeedJobNotificationHandler implements Runnable {
-
- private static final Logger LOGGER = Logger.getLogger(FeedJobNotificationHandler.class.getName());
-
- private final LinkedBlockingQueue<FeedEvent> inbox;
- private final Map<FeedConnectionId, List<IFeedLifecycleEventSubscriber>> eventSubscribers;
- private final Map<JobId, FeedJobInfo> jobInfos;
- private final Map<FeedId, FeedIntakeInfo> intakeJobInfos;
- private final Map<FeedConnectionId, FeedConnectJobInfo> connectJobInfos;
- private final Map<FeedId, Pair<FeedOperationCounter, List<IFeedJoint>>> feedPipeline;
-
- public FeedJobNotificationHandler(LinkedBlockingQueue<FeedEvent> inbox) {
- this.inbox = inbox;
- this.jobInfos = new HashMap<JobId, FeedJobInfo>();
- this.intakeJobInfos = new HashMap<FeedId, FeedIntakeInfo>();
- this.connectJobInfos = new HashMap<FeedConnectionId, FeedConnectJobInfo>();
- this.feedPipeline = new HashMap<FeedId, Pair<FeedOperationCounter, List<IFeedJoint>>>();
- this.eventSubscribers = new HashMap<FeedConnectionId, List<IFeedLifecycleEventSubscriber>>();
- }
-
- @Override
- public void run() {
- FeedEvent event = null;
- Thread.currentThread().setName("FeedJobNotificationHandler");
- while (true) {
- try {
- event = inbox.take();
- switch (event.eventKind) {
- case JOB_START:
- handleJobStartEvent(event);
- break;
- case JOB_FINISH:
- handleJobFinishEvent(event);
- break;
- case PARTITION_START:
- handlePartitionStart(event);
- break;
- default:
- LOGGER.log(Level.WARNING, "Unknown Feed Event");
- break;
- }
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
-
- public Collection<FeedIntakeInfo> getFeedIntakeInfos() {
- return intakeJobInfos.values();
- }
-
- public Collection<FeedConnectJobInfo> getFeedConnectInfos() {
- return connectJobInfos.values();
- }
-
- public synchronized void registerFeedJoint(IFeedJoint feedJoint, int numOfPrividers) {
- Pair<FeedOperationCounter, List<IFeedJoint>> feedJointsOnPipeline = feedPipeline
- .get(feedJoint.getOwnerFeedId());
-
- if (feedJointsOnPipeline == null) {
- feedJointsOnPipeline = new Pair<FeedOperationCounter, List<IFeedJoint>>(
- new FeedOperationCounter(numOfPrividers), new ArrayList<IFeedJoint>());
- feedPipeline.put(feedJoint.getOwnerFeedId(), feedJointsOnPipeline);
- feedJointsOnPipeline.second.add(feedJoint);
- } else {
- if (!feedJointsOnPipeline.second.contains(feedJoint)) {
- feedJointsOnPipeline.second.add(feedJoint);
- } else {
- throw new IllegalArgumentException("Feed joint " + feedJoint + " already registered");
- }
- }
- }
-
- public synchronized void registerFeedIntakeJob(FeedId feedId, JobId jobId, JobSpecification jobSpec)
- throws HyracksDataException {
- if (jobInfos.get(jobId) != null) {
- throw new IllegalStateException("Feed job already registered");
- }
-
- Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.containsKey(feedId) ? feedPipeline.get(feedId)
- : null;
- IFeedJoint intakeJoint = null;
- for (IFeedJoint joint : pair.second) {
- if (joint.getType().equals(IFeedJoint.FeedJointType.INTAKE)) {
- intakeJoint = joint;
- break;
- }
- }
-
- if (intakeJoint != null) {
- FeedIntakeInfo intakeJobInfo = new FeedIntakeInfo(jobId, FeedJobState.CREATED, FeedJobInfo.JobType.INTAKE,
- feedId, intakeJoint, jobSpec);
- pair.first.setFeedJobInfo(intakeJobInfo);
- intakeJobInfos.put(feedId, intakeJobInfo);
- jobInfos.put(jobId, intakeJobInfo);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registered feed intake [" + jobId + "]" + " for feed " + feedId);
- }
- } else {
- throw new HyracksDataException(
- "Could not register feed intake job [" + jobId + "]" + " for feed " + feedId);
- }
- }
-
- public synchronized void registerFeedCollectionJob(FeedId sourceFeedId, FeedConnectionId connectionId, JobId jobId,
- JobSpecification jobSpec, Map<String, String> feedPolicy) {
- if (jobInfos.get(jobId) != null) {
- throw new IllegalStateException("Feed job already registered");
- }
-
- List<IFeedJoint> feedJoints = feedPipeline.get(sourceFeedId).second;
- FeedConnectionId cid = null;
- IFeedJoint sourceFeedJoint = null;
- for (IFeedJoint joint : feedJoints) {
- cid = joint.getReceiver(connectionId);
- if (cid != null) {
- sourceFeedJoint = joint;
- break;
- }
- }
-
- if (cid != null) {
- FeedConnectJobInfo cInfo = new FeedConnectJobInfo(jobId, FeedJobState.CREATED, connectionId,
- sourceFeedJoint, null, jobSpec, feedPolicy);
- jobInfos.put(jobId, cInfo);
- connectJobInfos.put(connectionId, cInfo);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registered feed connection [" + jobId + "]" + " for feed " + connectionId);
- }
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Could not register feed collection job [" + jobId + "]" + " for feed connection "
- + connectionId);
- }
- }
-
- }
-
- public synchronized void deregisterFeedIntakeJob(JobId jobId) {
- if (jobInfos.get(jobId) == null) {
- throw new IllegalStateException(" Feed Intake job not registered ");
- }
-
- FeedIntakeInfo info = (FeedIntakeInfo) jobInfos.get(jobId);
- jobInfos.remove(jobId);
- intakeJobInfos.remove(info.getFeedId());
-
- if (!info.getState().equals(FeedJobState.UNDER_RECOVERY)) {
- List<IFeedJoint> joints = feedPipeline.get(info.getFeedId()).second;
- joints.remove(info.getIntakeFeedJoint());
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Deregistered feed intake job [" + jobId + "]");
- }
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Not removing feed joint as intake job is in " + FeedJobState.UNDER_RECOVERY + " state.");
- }
- }
-
- }
-
- private synchronized void handleJobStartEvent(FeedEvent message) throws Exception {
- FeedJobInfo jobInfo = jobInfos.get(message.jobId);
- switch (jobInfo.getJobType()) {
- case INTAKE:
- handleIntakeJobStartMessage((FeedIntakeInfo) jobInfo);
- break;
- case FEED_CONNECT:
- handleCollectJobStartMessage((FeedConnectJobInfo) jobInfo);
- break;
- }
-
- }
-
- private synchronized void handleJobFinishEvent(FeedEvent message) throws Exception {
- FeedJobInfo jobInfo = jobInfos.get(message.jobId);
- switch (jobInfo.getJobType()) {
- case INTAKE:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Intake Job finished for feed intake " + jobInfo.getJobId());
- }
- handleFeedIntakeJobFinishMessage((FeedIntakeInfo) jobInfo, message);
- break;
- case FEED_CONNECT:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Collect Job finished for " + jobInfo);
- }
- handleFeedCollectJobFinishMessage((FeedConnectJobInfo) jobInfo);
- break;
- }
- }
-
- private synchronized void handlePartitionStart(FeedEvent message) {
- FeedJobInfo jobInfo = jobInfos.get(message.jobId);
- switch (jobInfo.getJobType()) {
- case FEED_CONNECT:
- ((FeedConnectJobInfo) jobInfo).partitionStart();
- if (((FeedConnectJobInfo) jobInfo).collectionStarted()) {
- notifyFeedEventSubscribers(jobInfo, FeedLifecycleEvent.FEED_COLLECT_STARTED);
- }
- break;
- case INTAKE:
- Pair<FeedOperationCounter, List<IFeedJoint>> feedCounter = feedPipeline.get(message.feedId);
- feedCounter.first.setPartitionCount(feedCounter.first.getPartitionCount() - 1);;
- if (feedCounter.first.getPartitionCount() == 0) {
- ((FeedIntakeInfo) jobInfo).getIntakeFeedJoint().setState(State.ACTIVE);
- jobInfo.setState(FeedJobState.ACTIVE);
- notifyFeedEventSubscribers(jobInfo, FeedLifecycleEvent.FEED_INTAKE_STARTED);
- }
- break;
- default:
- break;
-
- }
- }
-
- private synchronized void handleIntakeJobStartMessage(FeedIntakeInfo intakeJobInfo) throws Exception {
- List<OperatorDescriptorId> intakeOperatorIds = new ArrayList<OperatorDescriptorId>();
- Map<OperatorDescriptorId, IOperatorDescriptor> operators = intakeJobInfo.getSpec().getOperatorMap();
- for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
- IOperatorDescriptor opDesc = entry.getValue();
- if (opDesc instanceof FeedIntakeOperatorDescriptor) {
- intakeOperatorIds.add(opDesc.getOperatorId());
- }
- }
-
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- JobInfo info = hcc.getJobInfo(intakeJobInfo.getJobId());
- List<String> intakeLocations = new ArrayList<String>();
- for (OperatorDescriptorId intakeOperatorId : intakeOperatorIds) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(intakeOperatorId);
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- intakeLocations.add(operatorLocations.get(i));
- }
- }
- // intakeLocations is an ordered list; element at position i corresponds to location of i'th instance of operator
- intakeJobInfo.setIntakeLocation(intakeLocations);
- }
-
- private void handleCollectJobStartMessage(FeedConnectJobInfo cInfo) throws RemoteException, ACIDException {
- // set locations of feed sub-operations (intake, compute, store)
- setLocations(cInfo);
- Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(cInfo.getConnectionId().getFeedId());
- // activate joints
- List<IFeedJoint> joints = pair.second;
- for (IFeedJoint joint : joints) {
- if (joint.getProvider().equals(cInfo.getConnectionId())) {
- joint.setState(State.ACTIVE);
- if (joint.getType().equals(IFeedJoint.FeedJointType.COMPUTE)) {
- cInfo.setComputeFeedJoint(joint);
- }
- }
- }
- cInfo.setState(FeedJobState.ACTIVE);
- }
-
- private void notifyFeedEventSubscribers(FeedJobInfo jobInfo, FeedLifecycleEvent event) {
- JobType jobType = jobInfo.getJobType();
- List<FeedConnectionId> impactedConnections = new ArrayList<FeedConnectionId>();
- if (jobType.equals(JobType.INTAKE)) {
- FeedId feedId = ((FeedIntakeInfo) jobInfo).getFeedId();
- for (FeedConnectionId connId : eventSubscribers.keySet()) {
- if (connId.getFeedId().equals(feedId)) {
- impactedConnections.add(connId);
- }
- }
- } else {
- impactedConnections.add(((FeedConnectJobInfo) jobInfo).getConnectionId());
- }
-
- for (FeedConnectionId connId : impactedConnections) {
- List<IFeedLifecycleEventSubscriber> subscribers = eventSubscribers.get(connId);
- if (subscribers != null && !subscribers.isEmpty()) {
- for (IFeedLifecycleEventSubscriber subscriber : subscribers) {
- subscriber.handleFeedEvent(event);
- }
- }
- if (event == FeedLifecycleEvent.FEED_COLLECT_ENDED) {
- eventSubscribers.remove(connId);
- }
- }
- }
-
- public synchronized void submitFeedConnectionRequest(IFeedJoint feedJoint, final FeedConnectionRequest request)
- throws Exception {
- List<String> locations = null;
- switch (feedJoint.getType()) {
- case INTAKE:
- FeedIntakeInfo intakeInfo = intakeJobInfos.get(feedJoint.getOwnerFeedId());
- locations = intakeInfo.getIntakeLocation();
- break;
- case COMPUTE:
- FeedConnectionId connectionId = feedJoint.getProvider();
- FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
- locations = cInfo.getComputeLocations();
- break;
- }
-
- SubscribeFeedWork work = new SubscribeFeedWork(locations.toArray(new String[] {}), request);
- FeedWorkManager.INSTANCE.submitWork(work, new SubscribeFeedWork.FeedSubscribeWorkEventListener());
- }
-
- public IFeedJoint getSourceFeedJoint(FeedConnectionId connectionId) {
- FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
- if (cInfo != null) {
- return cInfo.getSourceFeedJoint();
- }
- return null;
- }
-
- public Set<FeedConnectionId> getActiveFeedConnections() {
- Set<FeedConnectionId> activeConnections = new HashSet<FeedConnectionId>();
- for (FeedConnectJobInfo cInfo : connectJobInfos.values()) {
- if (cInfo.getState().equals(FeedJobState.ACTIVE)) {
- activeConnections.add(cInfo.getConnectionId());
- }
- }
- return activeConnections;
- }
-
- public synchronized boolean isFeedConnectionActive(FeedConnectionId connectionId,
- IFeedLifecycleEventSubscriber eventSubscriber) {
- boolean active = false;
- FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
- if (cInfo != null) {
- active = cInfo.getState().equals(FeedJobState.ACTIVE);
- }
- if (active) {
- registerFeedEventSubscriber(connectionId, eventSubscriber);
- }
- return active;
- }
-
- public void setJobState(FeedConnectionId connectionId, FeedJobState jobState) {
- FeedConnectJobInfo connectJobInfo = connectJobInfos.get(connectionId);
- connectJobInfo.setState(jobState);
- }
-
- public FeedJobState getFeedJobState(FeedConnectionId connectionId) {
- return connectJobInfos.get(connectionId).getState();
- }
-
- private synchronized void handleFeedIntakeJobFinishMessage(FeedIntakeInfo intakeInfo, FeedEvent message)
- throws Exception {
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- JobInfo info = hcc.getJobInfo(message.jobId);
- JobStatus status = info.getStatus();
- FeedId feedId = intakeInfo.getFeedId();
- Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(feedId);
- if (status.equals(JobStatus.FAILURE)) {
- pair.first.setFailedIngestion(true);
- }
- // remove feed joints
- deregisterFeedIntakeJob(message.jobId);
- // notify event listeners
- feedPipeline.remove(feedId);
- intakeJobInfos.remove(feedId);
- notifyFeedEventSubscribers(intakeInfo, pair.first.isFailedIngestion() ? FeedLifecycleEvent.FEED_INTAKE_FAILURE
- : FeedLifecycleEvent.FEED_INTAKE_ENDED);
- }
-
- private synchronized void handleFeedCollectJobFinishMessage(FeedConnectJobInfo cInfo) throws Exception {
- FeedConnectionId connectionId = cInfo.getConnectionId();
-
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- JobInfo info = hcc.getJobInfo(cInfo.getJobId());
- JobStatus status = info.getStatus();
- boolean failure = status != null && status.equals(JobStatus.FAILURE);
- FeedPolicyAccessor fpa = new FeedPolicyAccessor(cInfo.getFeedPolicy());
- boolean retainSubsription = cInfo.getState().equals(FeedJobState.UNDER_RECOVERY)
- || (failure && fpa.continueOnHardwareFailure());
-
- if (!retainSubsription) {
- IFeedJoint feedJoint = cInfo.getSourceFeedJoint();
- feedJoint.removeReceiver(connectionId);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(
- "Subscription " + cInfo.getConnectionId() + " completed successfully. Removed subscription");
- }
- }
-
- connectJobInfos.remove(connectionId);
- jobInfos.remove(cInfo.getJobId());
- // notify event listeners
- FeedLifecycleEvent event = failure ? FeedLifecycleEvent.FEED_COLLECT_FAILURE
- : FeedLifecycleEvent.FEED_COLLECT_ENDED;
- notifyFeedEventSubscribers(cInfo, event);
- }
-
- public boolean isRegisteredFeedJob(JobId jobId) {
- return jobInfos.get(jobId) != null;
- }
-
- public List<String> getFeedComputeLocations(FeedId feedId) {
- List<IFeedJoint> feedJoints = feedPipeline.get(feedId).second;
- for (IFeedJoint joint : feedJoints) {
- if (joint.getFeedJointKey().getFeedId().equals(feedId)) {
- return connectJobInfos.get(joint.getProvider()).getComputeLocations();
- }
- }
- return null;
- }
-
- public List<String> getFeedStorageLocations(FeedConnectionId connectionId) {
- return connectJobInfos.get(connectionId).getStorageLocations();
- }
-
- public List<String> getFeedCollectLocations(FeedConnectionId connectionId) {
- return connectJobInfos.get(connectionId).getCollectLocations();
- }
-
- public List<String> getFeedIntakeLocations(FeedId feedId) {
- return intakeJobInfos.get(feedId).getIntakeLocation();
- }
-
- public JobId getFeedCollectJobId(FeedConnectionId connectionId) {
- return connectJobInfos.get(connectionId).getJobId();
- }
-
- public synchronized void registerFeedEventSubscriber(FeedConnectionId connectionId,
- IFeedLifecycleEventSubscriber subscriber) {
- List<IFeedLifecycleEventSubscriber> subscribers = eventSubscribers.get(connectionId);
- if (subscribers == null) {
- subscribers = new ArrayList<IFeedLifecycleEventSubscriber>();
- eventSubscribers.put(connectionId, subscribers);
- }
- subscribers.add(subscriber);
- }
-
- public void deregisterFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
- List<IFeedLifecycleEventSubscriber> subscribers = eventSubscribers.get(connectionId);
- if (subscribers != null) {
- subscribers.remove(subscriber);
- }
- }
-
- // ============================
-
- public boolean isFeedPointAvailable(FeedJointKey feedJointKey) {
- List<IFeedJoint> joints = feedPipeline.containsKey(feedJointKey.getFeedId())
- ? feedPipeline.get(feedJointKey.getFeedId()).second : null;
- if (joints != null && !joints.isEmpty()) {
- for (IFeedJoint joint : joints) {
- if (joint.getFeedJointKey().equals(feedJointKey)) {
- return true;
- }
- }
- }
- return false;
- }
-
- public Collection<IFeedJoint> getFeedIntakeJoints() {
- List<IFeedJoint> intakeFeedPoints = new ArrayList<IFeedJoint>();
- for (FeedIntakeInfo info : intakeJobInfos.values()) {
- intakeFeedPoints.add(info.getIntakeFeedJoint());
- }
- return intakeFeedPoints;
- }
-
- public IFeedJoint getFeedJoint(FeedJointKey feedPointKey) {
- List<IFeedJoint> joints = feedPipeline.containsKey(feedPointKey.getFeedId())
- ? feedPipeline.get(feedPointKey.getFeedId()).second : null;
- if (joints != null && !joints.isEmpty()) {
- for (IFeedJoint joint : joints) {
- if (joint.getFeedJointKey().equals(feedPointKey)) {
- return joint;
- }
- }
- }
- return null;
- }
-
- public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
- IFeedJoint feedJoint = getFeedJoint(feedJointKey);
- if (feedJoint != null) {
- return feedJoint;
- } else {
- String jointKeyString = feedJointKey.getStringRep();
- List<IFeedJoint> jointsOnPipeline = feedPipeline.containsKey(feedJointKey.getFeedId())
- ? feedPipeline.get(feedJointKey.getFeedId()).second : null;
- IFeedJoint candidateJoint = null;
- if (jointsOnPipeline != null) {
- for (IFeedJoint joint : jointsOnPipeline) {
- if (jointKeyString.contains(joint.getFeedJointKey().getStringRep())) {
- if (candidateJoint == null) {
- candidateJoint = joint;
- } else if (joint.getFeedJointKey().getStringRep()
- .contains(candidateJoint.getFeedJointKey().getStringRep())) { // found feed point is a super set of the earlier find
- candidateJoint = joint;
- }
- }
- }
- }
- return candidateJoint;
- }
- }
-
- public JobSpecification getCollectJobSpecification(FeedConnectionId connectionId) {
- return connectJobInfos.get(connectionId).getSpec();
- }
-
- public IFeedJoint getFeedPoint(FeedId sourceFeedId, IFeedJoint.FeedJointType type) {
- List<IFeedJoint> joints = feedPipeline.get(sourceFeedId).second;
- for (IFeedJoint joint : joints) {
- if (joint.getType().equals(type)) {
- return joint;
- }
- }
- return null;
- }
-
- public FeedConnectJobInfo getFeedConnectJobInfo(FeedConnectionId connectionId) {
- return connectJobInfos.get(connectionId);
- }
-
- private void setLocations(FeedConnectJobInfo cInfo) {
- JobSpecification jobSpec = cInfo.getSpec();
-
- List<OperatorDescriptorId> collectOperatorIds = new ArrayList<OperatorDescriptorId>();
- List<OperatorDescriptorId> computeOperatorIds = new ArrayList<OperatorDescriptorId>();
- List<OperatorDescriptorId> storageOperatorIds = new ArrayList<OperatorDescriptorId>();
-
- Map<OperatorDescriptorId, IOperatorDescriptor> operators = jobSpec.getOperatorMap();
- for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
- IOperatorDescriptor opDesc = entry.getValue();
- IOperatorDescriptor actualOp = null;
- if (opDesc instanceof FeedMetaOperatorDescriptor) {
- actualOp = ((FeedMetaOperatorDescriptor) opDesc).getCoreOperator();
- } else {
- actualOp = opDesc;
- }
-
- if (actualOp instanceof AlgebricksMetaOperatorDescriptor) {
- AlgebricksMetaOperatorDescriptor op = ((AlgebricksMetaOperatorDescriptor) actualOp);
- IPushRuntimeFactory[] runtimeFactories = op.getPipeline().getRuntimeFactories();
- boolean computeOp = false;
- for (IPushRuntimeFactory rf : runtimeFactories) {
- if (rf instanceof AssignRuntimeFactory) {
- IConnectorDescriptor connDesc = jobSpec.getOperatorInputMap().get(op.getOperatorId()).get(0);
- IOperatorDescriptor sourceOp = jobSpec.getConnectorOperatorMap().get(connDesc.getConnectorId())
- .getLeft().getLeft();
- if (sourceOp instanceof FeedCollectOperatorDescriptor) {
- computeOp = true;
- break;
- }
- }
- }
- if (computeOp) {
- computeOperatorIds.add(entry.getKey());
- }
- } else if (actualOp instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
- storageOperatorIds.add(entry.getKey());
- } else if (actualOp instanceof FeedCollectOperatorDescriptor) {
- collectOperatorIds.add(entry.getKey());
- }
- }
-
- try {
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- JobInfo info = hcc.getJobInfo(cInfo.getJobId());
- List<String> collectLocations = new ArrayList<String>();
- for (OperatorDescriptorId collectOpId : collectOperatorIds) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(collectOpId);
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- collectLocations.add(operatorLocations.get(i));
- }
- }
-
- List<String> computeLocations = new ArrayList<String>();
- for (OperatorDescriptorId computeOpId : computeOperatorIds) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(computeOpId);
- if (operatorLocations != null) {
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- computeLocations.add(operatorLocations.get(i));
- }
- } else {
- computeLocations.clear();
- computeLocations.addAll(collectLocations);
- }
- }
-
- List<String> storageLocations = new ArrayList<String>();
- for (OperatorDescriptorId storageOpId : storageOperatorIds) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(storageOpId);
- if (operatorLocations == null) {
- continue;
- }
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- storageLocations.add(operatorLocations.get(i));
- }
- }
- cInfo.setCollectLocations(collectLocations);
- cInfo.setComputeLocations(computeLocations);
- cInfo.setStorageLocations(storageLocations);
-
- } catch (Exception e) {
- e.printStackTrace();
- }
-
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java
index 3f0bc54..0614d14 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java
@@ -23,15 +23,21 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.active.ActiveJobNotificationHandler;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.app.external.FeedWorkCollection.SubscribeFeedWork;
import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.management.FeedEventsListener;
import org.apache.asterix.external.feed.management.FeedJointKey;
+import org.apache.asterix.external.feed.management.FeedWorkManager;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
public class FeedJoint implements IFeedJoint {
+ private static final long serialVersionUID = 1L;
+
private static final Logger LOGGER = Logger.getLogger(FeedJoint.class.getName());
/** A unique key associated with the feed point **/
@@ -44,7 +50,7 @@
private final List<FeedConnectionId> receivers;
/** The feedId on which the feedPoint resides **/
- private final FeedId ownerFeedId;
+ private final EntityId ownerFeedId;
/** A list of feed subscription requests submitted for subscribing to the FeedPoint's data **/
private final List<FeedConnectionRequest> connectionRequests;
@@ -55,7 +61,7 @@
private FeedConnectionId provider;
- public FeedJoint(FeedJointKey key, FeedId ownerFeedId, FeedRuntimeType subscriptionLocation, FeedJointType type,
+ public FeedJoint(FeedJointKey key, EntityId ownerFeedId, FeedRuntimeType subscriptionLocation, FeedJointType type,
FeedConnectionId provider) {
this.key = key;
this.ownerFeedId = ownerFeedId;
@@ -106,10 +112,15 @@
private void handlePendingConnectionRequest() {
for (FeedConnectionRequest connectionRequest : connectionRequests) {
- FeedConnectionId connectionId = new FeedConnectionId(connectionRequest.getReceivingFeedId(),
- connectionRequest.getTargetDataset());
+ FeedConnectionId connectionId =
+ new FeedConnectionId(connectionRequest.getReceivingFeedId(), connectionRequest.getTargetDataset());
try {
- FeedLifecycleListener.INSTANCE.submitFeedConnectionRequest(this, connectionRequest);
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(connectionId.getFeedId());
+ SubscribeFeedWork work = new SubscribeFeedWork(
+ listener.getConnectionLocations(this, connectionRequest).toArray(new String[] {}),
+ connectionRequest);
+ FeedWorkManager.INSTANCE.submitWork(work, new SubscribeFeedWork.FeedSubscribeWorkEventListener());
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Submitted feed connection request " + connectionRequest + " at feed joint " + this);
}
@@ -155,7 +166,7 @@
}
@Override
- public FeedId getOwnerFeedId() {
+ public EntityId getOwnerFeedId() {
return ownerFeedId;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java
deleted file mode 100644
index b8435af..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java
+++ /dev/null
@@ -1,228 +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.app.external;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber;
-import org.apache.asterix.external.feed.api.IFeedLifecycleListener;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.feed.watch.FeedJobInfo.FeedJobState;
-import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-/**
- * A listener that subscribes to events associated with cluster membership
- * (nodes joining/leaving the cluster) and job lifecycle (start/end of a job).
- * Subscription to such events allows keeping track of feed ingestion jobs and
- * take any corrective action that may be required when a node involved in a
- * feed leaves the cluster.
- */
-public class FeedLifecycleListener implements IFeedLifecycleListener {
-
- private static final Logger LOGGER = Logger.getLogger(FeedLifecycleListener.class.getName());
- public static FeedLifecycleListener INSTANCE = new FeedLifecycleListener();
-
- private final LinkedBlockingQueue<FeedEvent> jobEventInbox;
- private final FeedJobNotificationHandler feedJobNotificationHandler;
- private final ExecutorService executorService;
-
- private FeedLifecycleListener() {
- this.jobEventInbox = new LinkedBlockingQueue<FeedEvent>();
- this.feedJobNotificationHandler = new FeedJobNotificationHandler(jobEventInbox);
- this.executorService = Executors.newCachedThreadPool();
- this.executorService.execute(feedJobNotificationHandler);
- }
-
- @Override
- public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
- if (feedJobNotificationHandler.isRegisteredFeedJob(jobId)) {
- jobEventInbox.add(new FeedEvent(jobId, FeedEvent.EventKind.JOB_START));
- }
- }
-
- @Override
- public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
- if (feedJobNotificationHandler.isRegisteredFeedJob(jobId)) {
- jobEventInbox.add(new FeedEvent(jobId, FeedEvent.EventKind.JOB_FINISH));
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
- }
- }
- }
-
- public FeedConnectJobInfo getFeedConnectJobInfo(FeedConnectionId connectionId) {
- return feedJobNotificationHandler.getFeedConnectJobInfo(connectionId);
- }
-
- /*
- * Traverse job specification to categorize job as a feed intake job or a feed collection job
- */
- @Override
- public void notifyJobCreation(JobId jobId, IActivityClusterGraphGeneratorFactory acggf) throws HyracksException {
- JobSpecification spec = acggf.getJobSpecification();
- FeedConnectionId feedConnectionId = null;
- Map<String, String> feedPolicy = null;
- for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
- if (opDesc instanceof FeedCollectOperatorDescriptor) {
- feedConnectionId = ((FeedCollectOperatorDescriptor) opDesc).getFeedConnectionId();
- feedPolicy = ((FeedCollectOperatorDescriptor) opDesc).getFeedPolicyProperties();
- feedJobNotificationHandler.registerFeedCollectionJob(
- ((FeedCollectOperatorDescriptor) opDesc).getSourceFeedId(), feedConnectionId, jobId, spec,
- feedPolicy);
- break;
- } else if (opDesc instanceof FeedIntakeOperatorDescriptor) {
- feedJobNotificationHandler.registerFeedIntakeJob(((FeedIntakeOperatorDescriptor) opDesc).getFeedId(),
- jobId, spec);
- break;
- }
- }
- }
-
- public void setJobState(FeedConnectionId connectionId, FeedJobState jobState) {
- feedJobNotificationHandler.setJobState(connectionId, jobState);
- }
-
- public FeedJobState getFeedJobState(FeedConnectionId connectionId) {
- return feedJobNotificationHandler.getFeedJobState(connectionId);
- }
-
- public static class FeedEvent {
- public JobId jobId;
- public FeedId feedId;
-
- public enum EventKind {
- JOB_START,
- JOB_FINISH,
- PARTITION_START
- }
-
- public EventKind eventKind;
-
- public FeedEvent(JobId jobId, EventKind eventKind) {
- this(jobId, eventKind, null);
- }
-
- public FeedEvent(JobId jobId, EventKind eventKind, FeedId feedId) {
- this.jobId = jobId;
- this.eventKind = eventKind;
- this.feedId = feedId;
- }
- }
-
- public void submitFeedConnectionRequest(IFeedJoint feedPoint, FeedConnectionRequest subscriptionRequest)
- throws Exception {
- feedJobNotificationHandler.submitFeedConnectionRequest(feedPoint, subscriptionRequest);
- }
-
- @Override
- public List<FeedConnectionId> getActiveFeedConnections(FeedId feedId) {
- List<FeedConnectionId> connections = new ArrayList<FeedConnectionId>();
- Collection<FeedConnectionId> activeConnections = feedJobNotificationHandler.getActiveFeedConnections();
- if (feedId != null) {
- for (FeedConnectionId connectionId : activeConnections) {
- if (connectionId.getFeedId().equals(feedId)) {
- connections.add(connectionId);
- }
- }
- } else {
- connections.addAll(activeConnections);
- }
- return connections;
- }
-
- @Override
- public List<String> getComputeLocations(FeedId feedId) {
- return feedJobNotificationHandler.getFeedComputeLocations(feedId);
- }
-
- @Override
- public List<String> getIntakeLocations(FeedId feedId) {
- return feedJobNotificationHandler.getFeedIntakeLocations(feedId);
- }
-
- @Override
- public List<String> getStoreLocations(FeedConnectionId feedConnectionId) {
- return feedJobNotificationHandler.getFeedStorageLocations(feedConnectionId);
- }
-
- @Override
- public List<String> getCollectLocations(FeedConnectionId feedConnectionId) {
- return feedJobNotificationHandler.getFeedCollectLocations(feedConnectionId);
- }
-
- @Override
- public synchronized boolean isFeedConnectionActive(FeedConnectionId connectionId,
- IFeedLifecycleEventSubscriber eventSubscriber) {
- return feedJobNotificationHandler.isFeedConnectionActive(connectionId, eventSubscriber);
- }
-
- @Override
- public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
- return feedJobNotificationHandler.getAvailableFeedJoint(feedJointKey);
- }
-
- @Override
- public boolean isFeedJointAvailable(FeedJointKey feedJointKey) {
- return feedJobNotificationHandler.isFeedPointAvailable(feedJointKey);
- }
-
- public void registerFeedJoint(IFeedJoint feedJoint, int numOfPrividers) {
- feedJobNotificationHandler.registerFeedJoint(feedJoint, numOfPrividers);
- }
-
- public IFeedJoint getFeedJoint(FeedJointKey feedJointKey) {
- return feedJobNotificationHandler.getFeedJoint(feedJointKey);
- }
-
- @Override
- public void registerFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
- feedJobNotificationHandler.registerFeedEventSubscriber(connectionId, subscriber);
- }
-
- @Override
- public void deregisterFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
- feedJobNotificationHandler.deregisterFeedEventSubscriber(connectionId, subscriber);
-
- }
-
- public synchronized void notifyPartitionStart(FeedId feedId, JobId jobId) {
- jobEventInbox.add(new FeedEvent(jobId, FeedEvent.EventKind.PARTITION_START, feedId));
- }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
index 2de0266..aab1aa3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
@@ -23,20 +23,22 @@
import java.util.Set;
import java.util.TreeSet;
+import org.apache.asterix.active.ActiveJobNotificationHandler;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveMessage;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.api.IFeedMessage;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.management.FeedEventsListener;
import org.apache.asterix.external.feed.message.EndFeedMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
import org.apache.asterix.external.operators.FeedMessageOperatorDescriptor;
import org.apache.asterix.external.util.FeedConstants;
import org.apache.asterix.external.util.FeedUtils;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.asterix.file.JobSpecificationUtils;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.metadata.entities.Feed;
@@ -75,8 +77,8 @@
IAdapterFactory adapterFactory = null;
IOperatorDescriptor feedIngestor;
AlgebricksPartitionConstraint ingesterPc;
- Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t = metadataProvider
- .buildFeedIntakeRuntime(spec, primaryFeed, policyAccessor);
+ Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t =
+ metadataProvider.buildFeedIntakeRuntime(spec, primaryFeed, policyAccessor);
feedIngestor = t.first;
ingesterPc = t.second;
adapterFactory = t.third;
@@ -101,7 +103,9 @@
List<String> locations = null;
FeedRuntimeType sourceRuntimeType;
try {
- FeedConnectJobInfo cInfo = FeedLifecycleListener.INSTANCE.getFeedConnectJobInfo(connectionId);
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(connectionId.getFeedId());
+ FeedConnectJobInfo cInfo = listener.getFeedConnectJobInfo(connectionId);
IFeedJoint sourceFeedJoint = cInfo.getSourceFeedJoint();
IFeedJoint computeFeedJoint = cInfo.getComputeFeedJoint();
@@ -136,20 +140,20 @@
}
private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildSendFeedMessageRuntime(
- JobSpecification jobSpec, FeedConnectionId feedConenctionId, IFeedMessage feedMessage,
+ JobSpecification jobSpec, FeedConnectionId feedConenctionId, IActiveMessage feedMessage,
Collection<String> locations) throws AlgebricksException {
- AlgebricksPartitionConstraint partitionConstraint = new AlgebricksAbsolutePartitionConstraint(
- locations.toArray(new String[] {}));
- FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, feedConenctionId,
- feedMessage);
+ AlgebricksPartitionConstraint partitionConstraint =
+ new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[] {}));
+ FeedMessageOperatorDescriptor feedMessenger =
+ new FeedMessageOperatorDescriptor(jobSpec, feedConenctionId, feedMessage);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, partitionConstraint);
}
private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
JobSpecification jobSpec, FeedConnectionId feedConenctionId, List<String> locations,
- FeedRuntimeType sourceFeedRuntimeType, boolean completeDisconnection, FeedId sourceFeedId)
+ FeedRuntimeType sourceFeedRuntimeType, boolean completeDisconnection, EntityId sourceFeedId)
throws AlgebricksException {
- IFeedMessage feedMessage = new EndFeedMessage(feedConenctionId, sourceFeedRuntimeType, sourceFeedId,
+ IActiveMessage feedMessage = new EndFeedMessage(feedConenctionId, sourceFeedRuntimeType, sourceFeedId,
completeDisconnection, EndFeedMessage.EndMessageType.DISCONNECT_FEED);
return buildSendFeedMessageRuntime(jobSpec, feedConenctionId, feedMessage, locations);
}
@@ -161,12 +165,12 @@
for (String node : allCluster.getLocations()) {
nodes.add(node);
}
- AlgebricksAbsolutePartitionConstraint locations = new AlgebricksAbsolutePartitionConstraint(
- nodes.toArray(new String[nodes.size()]));
- FileSplit[] feedLogFileSplits = FeedUtils.splitsForAdapter(feed.getDataverseName(), feed.getFeedName(),
- locations);
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = StoragePathUtil
- .splitProviderAndPartitionConstraints(feedLogFileSplits);
+ AlgebricksAbsolutePartitionConstraint locations =
+ new AlgebricksAbsolutePartitionConstraint(nodes.toArray(new String[nodes.size()]));
+ FileSplit[] feedLogFileSplits =
+ FeedUtils.splitsForAdapter(feed.getDataverseName(), feed.getFeedName(), locations);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ StoragePathUtil.splitProviderAndPartitionConstraints(feedLogFileSplits);
FileRemoveOperatorDescriptor frod = new FileRemoveOperatorDescriptor(spec, splitsAndConstraint.first, true);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, frod, splitsAndConstraint.second);
spec.addRoot(frod);
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 7be79fd..ba15fb1 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
@@ -82,8 +82,8 @@
//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);
@@ -119,7 +119,6 @@
LOGGER.info(" Feed subscription request " + ((SubscribeFeedWork) work).request + " completed ");
}
}
-
}
public FeedConnectionRequest getRequest() {
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/aql/translator/QueryTranslator.java
index 295b308..4df4468 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/aql/translator/QueryTranslator.java
@@ -37,11 +37,14 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.active.ActiveJobNotificationHandler;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveEntityEventsListener;
import org.apache.asterix.api.common.APIFramework;
import org.apache.asterix.api.common.SessionConfig;
import org.apache.asterix.app.external.ExternalIndexingOperations;
import org.apache.asterix.app.external.FeedJoint;
-import org.apache.asterix.app.external.FeedLifecycleListener;
import org.apache.asterix.app.external.FeedOperations;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -61,15 +64,17 @@
import org.apache.asterix.external.feed.api.IFeedJoint.FeedJointType;
import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber;
import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber.FeedLifecycleEvent;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.management.FeedEventsListener;
import org.apache.asterix.external.feed.management.FeedJointKey;
import org.apache.asterix.external.feed.management.FeedLifecycleEventSubscriber;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.watch.FeedActivity.FeedActivityDetails;
+import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
+import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.asterix.file.DatasetOperations;
import org.apache.asterix.file.DataverseOperations;
import org.apache.asterix.file.IndexOperations;
@@ -134,6 +139,7 @@
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;
@@ -195,7 +201,7 @@
*/
public class QueryTranslator extends AbstractLangTranslator {
- private static Logger LOGGER = Logger.getLogger(QueryTranslator.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(QueryTranslator.class.getName());
private enum ProgressState {
NO_PROGRESS,
@@ -226,9 +232,9 @@
}
private List<FunctionDecl> getDeclaredFunctions(List<Statement> statements) {
- List<FunctionDecl> functionDecls = new ArrayList<FunctionDecl>();
+ List<FunctionDecl> functionDecls = new ArrayList<>();
for (Statement st : statements) {
- if (st.getKind().equals(Statement.Kind.FUNCTION_DECL)) {
+ if (st.getKind() == Statement.FUNCTION_DECL) {
functionDecls.add((FunctionDecl) st);
}
}
@@ -258,175 +264,138 @@
FileSplit outputFile = null;
IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
- Map<String, String> config = new HashMap<String, String>();
-
- for (Statement stmt : statements) {
- if (sessionConfig.is(SessionConfig.FORMAT_HTML)) {
- sessionConfig.out().println(APIFramework.HTML_STATEMENT_SEPARATOR);
+ Map<String, String> config = new HashMap<>();
+ /* Since the system runs a large number of threads, when HTTP requests don't return, it becomes difficult to
+ * find the thread running the request to determine where it has stopped.
+ * Setting the thread name helps make that easier
+ */
+ String threadName = Thread.currentThread().getName();
+ Thread.currentThread().setName(QueryTranslator.class.getSimpleName());
+ try {
+ for (Statement stmt : statements) {
+ if (sessionConfig.is(SessionConfig.FORMAT_HTML)) {
+ sessionConfig.out().println(APIFramework.HTML_STATEMENT_SEPARATOR);
+ }
+ validateOperation(activeDefaultDataverse, stmt);
+ rewriteStatement(stmt); // Rewrite the statement's AST.
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse);
+ metadataProvider.setWriterFactory(writerFactory);
+ metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
+ metadataProvider.setOutputFile(outputFile);
+ metadataProvider.setConfig(config);
+ switch (stmt.getKind()) {
+ case Statement.SET:
+ handleSetStatement(stmt, config);
+ break;
+ case Statement.DATAVERSE_DECL:
+ activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt);
+ break;
+ case Statement.CREATE_DATAVERSE:
+ handleCreateDataverseStatement(metadataProvider, stmt);
+ break;
+ case Statement.DATASET_DECL:
+ handleCreateDatasetStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.CREATE_INDEX:
+ handleCreateIndexStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.TYPE_DECL:
+ handleCreateTypeStatement(metadataProvider, stmt);
+ break;
+ case Statement.NODEGROUP_DECL:
+ handleCreateNodeGroupStatement(metadataProvider, stmt);
+ break;
+ case Statement.DATAVERSE_DROP:
+ handleDataverseDropStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.DATASET_DROP:
+ handleDatasetDropStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.INDEX_DROP:
+ handleIndexDropStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.TYPE_DROP:
+ handleTypeDropStatement(metadataProvider, stmt);
+ break;
+ case Statement.NODEGROUP_DROP:
+ handleNodegroupDropStatement(metadataProvider, stmt);
+ break;
+ case Statement.CREATE_FUNCTION:
+ handleCreateFunctionStatement(metadataProvider, stmt);
+ break;
+ case Statement.FUNCTION_DROP:
+ handleFunctionDropStatement(metadataProvider, stmt);
+ break;
+ case Statement.LOAD:
+ handleLoadStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.INSERT:
+ case Statement.UPSERT:
+ handleInsertUpsertStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.DELETE:
+ handleDeleteStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.CREATE_PRIMARY_FEED:
+ case Statement.CREATE_SECONDARY_FEED:
+ handleCreateFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.DROP_FEED:
+ handleDropFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.DROP_FEED_POLICY:
+ handleDropFeedPolicyStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.CONNECT_FEED:
+ handleConnectFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.DISCONNECT_FEED:
+ handleDisconnectFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.SUBSCRIBE_FEED:
+ handleSubscribeFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.CREATE_FEED_POLICY:
+ handleCreateFeedPolicyStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.QUERY:
+ metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+ metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
+ || resultDelivery == ResultDelivery.ASYNC_DEFERRED);
+ handleQuery(metadataProvider, (Query) stmt, hcc, hdc, resultDelivery, stats);
+ break;
+ case Statement.COMPACT:
+ handleCompactStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.EXTERNAL_DATASET_REFRESH:
+ handleExternalDatasetRefreshStatement(metadataProvider, stmt, hcc);
+ break;
+ case Statement.WRITE:
+ Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(stmt);
+ writerFactory = (result.first != null) ? result.first : writerFactory;
+ outputFile = result.second;
+ break;
+ case Statement.RUN:
+ handleRunStatement(metadataProvider, stmt, hcc);
+ break;
+ default:
+ // Default should delegate unknown statement to extension-manager
+ break;
+ }
}
- validateOperation(activeDefaultDataverse, stmt);
- rewriteStatement(stmt); // Rewrite the statement's AST.
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse);
- metadataProvider.setWriterFactory(writerFactory);
- metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
- metadataProvider.setOutputFile(outputFile);
- metadataProvider.setConfig(config);
- switch (stmt.getKind()) {
- case SET: {
- handleSetStatement(metadataProvider, stmt, config);
- break;
- }
- case DATAVERSE_DECL: {
- activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt);
- break;
- }
- case CREATE_DATAVERSE: {
- handleCreateDataverseStatement(metadataProvider, stmt);
- break;
- }
- case DATASET_DECL: {
- handleCreateDatasetStatement(metadataProvider, stmt, hcc);
- break;
- }
- case CREATE_INDEX: {
- handleCreateIndexStatement(metadataProvider, stmt, hcc);
- break;
- }
- case TYPE_DECL: {
- handleCreateTypeStatement(metadataProvider, stmt);
- break;
- }
- case NODEGROUP_DECL: {
- handleCreateNodeGroupStatement(metadataProvider, stmt);
- break;
- }
- case DATAVERSE_DROP: {
- handleDataverseDropStatement(metadataProvider, stmt, hcc);
- break;
- }
- case DATASET_DROP: {
- handleDatasetDropStatement(metadataProvider, stmt, hcc);
- break;
- }
- case INDEX_DROP: {
- handleIndexDropStatement(metadataProvider, stmt, hcc);
- break;
- }
- case TYPE_DROP: {
- handleTypeDropStatement(metadataProvider, stmt);
- break;
- }
- case NODEGROUP_DROP: {
- handleNodegroupDropStatement(metadataProvider, stmt);
- break;
- }
-
- case CREATE_FUNCTION: {
- handleCreateFunctionStatement(metadataProvider, stmt);
- break;
- }
-
- case FUNCTION_DROP: {
- handleFunctionDropStatement(metadataProvider, stmt);
- break;
- }
-
- case LOAD: {
- handleLoadStatement(metadataProvider, stmt, hcc);
- break;
- }
- case INSERT:
- case UPSERT: {
- handleInsertUpsertStatement(metadataProvider, stmt, hcc);
- break;
- }
- case DELETE: {
- handleDeleteStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case CREATE_PRIMARY_FEED:
- case CREATE_SECONDARY_FEED: {
- handleCreateFeedStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case DROP_FEED: {
- handleDropFeedStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case DROP_FEED_POLICY: {
- handleDropFeedPolicyStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case CONNECT_FEED: {
- handleConnectFeedStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case DISCONNECT_FEED: {
- handleDisconnectFeedStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case SUBSCRIBE_FEED: {
- handleSubscribeFeedStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case CREATE_FEED_POLICY: {
- handleCreateFeedPolicyStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case QUERY: {
- metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
- metadataProvider.setResultAsyncMode(
- resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.ASYNC_DEFERRED);
- handleQuery(metadataProvider, (Query) stmt, hcc, hdc, resultDelivery, stats);
- break;
- }
-
- case COMPACT: {
- handleCompactStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case EXTERNAL_DATASET_REFRESH: {
- handleExternalDatasetRefreshStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- case WRITE: {
- Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(metadataProvider, stmt);
- if (result.first != null) {
- writerFactory = result.first;
- }
- outputFile = result.second;
- break;
- }
-
- case RUN: {
- handleRunStatement(metadataProvider, stmt, hcc);
- break;
- }
-
- default:
- break;
- }
+ } finally {
+ Thread.currentThread().setName(threadName);
}
}
- private void handleSetStatement(AqlMetadataProvider metadataProvider, Statement stmt, Map<String, String> config) {
+ private 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(AqlMetadataProvider metadataProvider, Statement stmt)
+ private Pair<IAWriterFactory, FileSplit> handleWriteStatement(Statement stmt)
throws InstantiationException, IllegalAccessException, ClassNotFoundException {
WriteStatement ws = (WriteStatement) stmt;
File f = new File(ws.getFileName());
@@ -435,7 +404,7 @@
if (ws.getWriterClassName() != null) {
writerFactory = (IAWriterFactory) Class.forName(ws.getWriterClassName()).newInstance();
}
- return new Pair<IAWriterFactory, FileSplit>(writerFactory, outputFile);
+ return new Pair<>(writerFactory, outputFile);
}
private Dataverse handleUseDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt)
@@ -497,8 +466,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.");
}
@@ -566,8 +535,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;
@@ -576,7 +545,7 @@
validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx, false);
}
switch (dd.getDatasetType()) {
- case INTERNAL: {
+ case INTERNAL:
IAType itemType = dt.getDatatype();
if (itemType.getTypeTag() != ATypeTag.RECORD) {
throw new AlgebricksException("Dataset type has to be a record type.");
@@ -591,10 +560,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,
@@ -617,16 +586,15 @@
InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
keySourceIndicators, partitioningTypes, autogenerated, filterField, temp);
break;
- }
- case EXTERNAL: {
+ case EXTERNAL:
String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
datasetDetails = new ExternalDatasetDetails(adapter, properties, new Date(),
ExternalDatasetTransactionState.COMMIT);
break;
- }
-
+ default:
+ throw new AsterixException("Unknown datatype " + dd.getDatasetType());
}
// #. initialize DatasetIdFactory if it is not initialized.
@@ -642,10 +610,10 @@
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);
@@ -719,24 +687,21 @@
}
private void validateIfResourceIsActiveInFeed(String dataverseName, String datasetName) throws AsterixException {
- List<FeedConnectionId> activeFeedConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(null);
- boolean resourceInUse = false;
- StringBuilder builder = new StringBuilder();
-
- if (activeFeedConnections != null && !activeFeedConnections.isEmpty()) {
- for (FeedConnectionId connId : activeFeedConnections) {
- if (connId.getDatasetName().equals(datasetName)) {
- resourceInUse = true;
- builder.append(connId + "\n");
+ StringBuilder builder = null;
+ IActiveEntityEventsListener[] listeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
+ for (IActiveEntityEventsListener listener : listeners) {
+ if (listener instanceof FeedEventsListener
+ && ((FeedEventsListener) listener).isConnectedToDataset(datasetName)) {
+ if (builder == null) {
+ builder = new StringBuilder();
}
+ builder.append(listener.getEntityId() + "\n");
}
}
-
- if (resourceInUse) {
- throw new AsterixException("Dataset " + datasetName + " is currently being "
+ if (builder != null) {
+ throw new AsterixException("Dataset " + dataverseName + "." + datasetName + " is currently being "
+ "fed into by the following feed(s).\n" + builder.toString() + "\n" + "Operation not supported");
}
-
}
private String getNodeGroupName(Identifier ngNameId, DatasetDecl dd, String dataverse) {
@@ -747,13 +712,13 @@
if (hintValue == null) {
return MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME;
} else {
- return (dataverse + ":" + dd.getName().getValue());
+ return dataverse + ":" + dd.getName().getValue();
}
}
private String configureNodegroupForDataset(DatasetDecl dd, String dataverse, MetadataTransactionContext mdTxnCtx)
throws AsterixException {
- int nodegroupCardinality = -1;
+ int nodegroupCardinality;
String nodegroupName;
String hintValue = dd.getHints().get(DatasetNodegroupCardinalityHint.NAME);
if (hintValue == null) {
@@ -849,8 +814,8 @@
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) {
@@ -877,8 +842,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);
}
@@ -1079,8 +1044,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);
@@ -1100,8 +1065,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;
@@ -1223,32 +1188,21 @@
throw new AlgebricksException("There is no dataverse with this name " + dataverseName + ".");
}
}
-
// # disconnect all feeds from any datasets in the dataverse.
- List<FeedConnectionId> activeFeedConnections = FeedLifecycleListener.INSTANCE
- .getActiveFeedConnections(null);
- DisconnectFeedStatement disStmt = null;
+ IActiveEntityEventsListener[] activeListeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
Identifier dvId = new Identifier(dataverseName);
- for (FeedConnectionId connection : activeFeedConnections) {
- FeedId feedId = connection.getFeedId();
- if (feedId.getDataverse().equals(dataverseName)) {
- disStmt = new DisconnectFeedStatement(dvId, new Identifier(feedId.getFeedName()),
- new Identifier(connection.getDatasetName()));
- try {
- handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Disconnected feed " + feedId.getFeedName() + " from dataset "
- + connection.getDatasetName());
- }
- } catch (Exception exception) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to disconnect feed " + feedId.getFeedName() + " from dataset "
- + connection.getDatasetName() + ". Encountered exception " + exception);
- }
+ for (IActiveEntityEventsListener listener : activeListeners) {
+ EntityId activeEntityId = listener.getEntityId();
+ if (activeEntityId.getExtensionName().equals(Feed.EXTENSION_NAME)
+ && activeEntityId.getDataverse().equals(dataverseName)) {
+ FeedEventsListener feedEventListener = (FeedEventsListener) listener;
+ FeedConnectionId[] connections = feedEventListener.getConnections();
+ for (FeedConnectionId conn : connections) {
+ disconnectFeedBeforeDelete(dvId, activeEntityId, conn, metadataProvider, hcc);
}
// prepare job to remove feed log storage
jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(
- MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedId.getFeedName())));
+ MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, activeEntityId.getEntityName())));
}
}
@@ -1258,8 +1212,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,
@@ -1273,8 +1227,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,
@@ -1358,6 +1312,24 @@
}
}
+ private 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()));
+ try {
+ handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disconnected feed " + activeEntityId.getEntityName() + " from dataset "
+ + conn.getDatasetName());
+ }
+ } catch (Exception exception) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to disconnect feed " + activeEntityId.getEntityName() + " from dataset "
+ + conn.getDatasetName() + ". Encountered exception " + exception);
+ }
+ }
+ }
+
private void handleDatasetDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
DropStatement stmtDelete = (DropStatement) stmt;
@@ -1387,20 +1359,23 @@
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.
- List<FeedConnectionId> feedConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(null);
- if (feedConnections != null && !feedConnections.isEmpty()) {
- for (FeedConnectionId connection : feedConnections) {
- Pair<JobSpecification, Boolean> p = FeedOperations.buildDisconnectFeedJobSpec(metadataProvider,
- connection);
- disconnectJobList.put(connection, p);
+ 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 " + connection.getFeedId().getFeedName() + " from dataset "
- + datasetName + " as dataset is being dropped");
+ 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,
- connection.getFeedId().getDataverse(), connection.getFeedId().getFeedName())));
+ jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(
+ MetadataManager.INSTANCE.getFeed(mdTxnCtx, connectionId.getFeedId().getDataverse(),
+ connectionId.getFeedId().getEntityName())));
}
}
@@ -1555,22 +1530,21 @@
throw new AlgebricksException(
"There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
}
-
- List<FeedConnectionId> feedConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(null);
- boolean resourceInUse = false;
- if (feedConnections != null && !feedConnections.isEmpty()) {
- StringBuilder builder = new StringBuilder();
- for (FeedConnectionId connection : feedConnections) {
- if (connection.getDatasetName().equals(datasetName)) {
- resourceInUse = true;
- builder.append(connection + "\n");
+ IActiveEntityEventsListener[] listeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
+ StringBuilder builder = null;
+ for (IActiveEntityEventsListener listener : listeners) {
+ if (listener.getEntityId().getExtensionName().equals(Feed.EXTENSION_NAME)
+ && ((FeedEventsListener) listener).isConnectedToDataset(datasetName)) {
+ if (builder == null) {
+ builder = new StringBuilder();
}
+ builder.append(new FeedConnectionId(listener.getEntityId(), datasetName) + "\n");
}
- if (resourceInUse) {
- throw new AsterixException(
- "Dataset" + datasetName + " is currently being fed into by the following feeds " + "."
- + builder.toString() + "\nOperation not supported.");
- }
+ }
+ if (builder != null) {
+ throw new AsterixException(
+ "Dataset" + datasetName + " is currently being fed into by the following feeds " + "."
+ + builder.toString() + "\nOperation not supported.");
}
if (ds.getDatasetType() == DatasetType.INTERNAL) {
@@ -1628,8 +1602,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
@@ -1846,11 +1820,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) {
@@ -1882,11 +1856,11 @@
metadataProvider.setWriteTransaction(true);
CompiledInsertStatement clfrqs = null;
switch (stmtInsertUpsert.getKind()) {
- case INSERT:
+ case Statement.INSERT:
clfrqs = new CompiledInsertStatement(dataverseName, stmtInsertUpsert.getDatasetName().getValue(),
query, stmtInsertUpsert.getVarCounter());
break;
- case UPSERT:
+ case Statement.UPSERT:
clfrqs = new CompiledUpsertStatement(dataverseName, stmtInsertUpsert.getDatasetName().getValue(),
query, stmtInsertUpsert.getVarCounter());
break;
@@ -1957,8 +1931,8 @@
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,
@@ -1989,13 +1963,13 @@
}
switch (stmt.getKind()) {
- case CREATE_PRIMARY_FEED:
+ case Statement.CREATE_PRIMARY_FEED:
CreatePrimaryFeedStatement cpfs = (CreatePrimaryFeedStatement) stmt;
String adaptorName = cpfs.getAdaptorName();
feed = new Feed(dataverseName, feedName, cfs.getAppliedFunction(), FeedType.PRIMARY, feedName,
adaptorName, cpfs.getAdaptorConfiguration());
break;
- case CREATE_SECONDARY_FEED:
+ case Statement.CREATE_SECONDARY_FEED:
CreateSecondaryFeedStatement csfs = (CreateSecondaryFeedStatement) stmt;
feed = new Feed(dataverseName, feedName, csfs.getAppliedFunction(), FeedType.SECONDARY,
csfs.getSourceFeedName(), null, null);
@@ -2026,8 +2000,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);
@@ -2094,19 +2068,19 @@
return;
}
- FeedId feedId = new FeedId(dataverseName, feedName);
- List<FeedConnectionId> activeConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(feedId);
- if (activeConnections != null && !activeConnections.isEmpty()) {
+ EntityId feedId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
+ FeedEventsListener listener =
+ (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(feedId);
+ if (listener != null) {
StringBuilder builder = new StringBuilder();
- for (FeedConnectionId connectionId : activeConnections) {
+ for (FeedConnectionId connectionId : listener.getConnections()) {
builder.append(connectionId.getDatasetName() + "\n");
}
-
throw new AlgebricksException("Feed " + feedId
+ " is currently active and connected to the following dataset(s) \n" + builder.toString());
} else {
JobSpecification spec = FeedOperations.buildRemoveFeedStorageJob(
- MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getFeedName()));
+ MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName()));
JobUtils.runJob(hcc, spec, true);
MetadataManager.INSTANCE.dropFeed(mdTxnCtx, dataverseName, feedName);
}
@@ -2160,61 +2134,62 @@
String dataverseName = getActiveDataverse(cfs.getDataverseName());
String feedName = cfs.getFeedName();
String datasetName = cfs.getDatasetName().getValue();
-
boolean bActiveTxn = true;
-
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
boolean subscriberRegistered = false;
IFeedLifecycleEventSubscriber eventSubscriber = new FeedLifecycleEventSubscriber();
FeedConnectionId feedConnId = null;
-
+ EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, cfs.getFeedName());
+ FeedEventsListener listener =
+ (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
MetadataLockManager.INSTANCE.connectFeedBegin(dataverseName, dataverseName + "." + datasetName,
dataverseName + "." + feedName);
try {
metadataProvider.setWriteTransaction(true);
-
CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(),
cfs.getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
-
FeedMetadataUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(),
metadataProvider.getMetadataTxnContext());
-
Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName(),
metadataProvider.getMetadataTxnContext());
-
feedConnId = new FeedConnectionId(dataverseName, cfs.getFeedName(), cfs.getDatasetName().getValue());
-
- subscriberRegistered = FeedLifecycleListener.INSTANCE.isFeedConnectionActive(feedConnId, eventSubscriber);
+ if (listener != null) {
+ subscriberRegistered = listener.isFeedConnectionActive(feedConnId, eventSubscriber);
+ }
if (subscriberRegistered) {
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;
- FeedLifecycleListener.INSTANCE.registerFeedEventSubscriber(feedConnId, eventSubscriber);
+ listener.registerFeedEventSubscriber(eventSubscriber);
subscriberRegistered = true;
if (createFeedIntakeJob) {
- FeedId feedId = connectionRequest.getFeedJointKey().getFeedId();
- Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(),
- feedId.getFeedName());
- Pair<JobSpecification, IAdapterFactory> pair = FeedOperations.buildFeedIntakeJobSpec(primaryFeed,
- metadataProvider, policyAccessor);
+ EntityId feedId = connectionRequest.getFeedJointKey().getFeedId();
+ 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;
for (IFeedJoint fj : triple.third) {
- FeedLifecycleListener.INSTANCE.registerFeedJoint(fj, numOfPrividers);
+ listener.registerFeedJoint(fj, numOfPrividers);
}
+ FeedIntakeInfo activeJob = new FeedIntakeInfo(null, ActivityState.ACTIVE, feed.getFeedId(),
+ 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
@@ -2227,7 +2202,7 @@
eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_INTAKE_STARTED);
} else {
for (IFeedJoint fj : triple.third) {
- FeedLifecycleListener.INSTANCE.registerFeedJoint(fj, 0);
+ listener.registerFeedJoint(fj, 0);
}
}
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2245,7 +2220,7 @@
MetadataLockManager.INSTANCE.connectFeedEnd(dataverseName, dataverseName + "." + datasetName,
dataverseName + "." + feedName);
if (subscriberRegistered) {
- FeedLifecycleListener.INSTANCE.deregisterFeedEventSubscriber(feedConnId, eventSubscriber);
+ listener.deregisterFeedEventSubscriber(eventSubscriber);
}
}
}
@@ -2260,27 +2235,33 @@
* @param feedPolicy
* @param mdTxnCtx
* @return
- * @throws MetadataException
+ * @throws AsterixException
*/
private Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> getFeedConnectionRequest(String dataverse,
Feed feed, String dataset, FeedPolicyEntity feedPolicy, MetadataTransactionContext mdTxnCtx)
- throws MetadataException {
+ throws AsterixException {
IFeedJoint sourceFeedJoint = null;
FeedConnectionRequest request = null;
List<String> functionsToApply = new ArrayList<String>();
boolean needIntakeJob = false;
List<IFeedJoint> jointsToRegister = new ArrayList<IFeedJoint>();
FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), dataset);
-
FeedRuntimeType connectionLocation = null;
FeedJointKey feedJointKey = getFeedJointKey(feed, mdTxnCtx);
- boolean isFeedJointAvailable = FeedLifecycleListener.INSTANCE.isFeedJointAvailable(feedJointKey);
+ EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverse, feed.getFeedName());
+ FeedEventsListener listener =
+ (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
+ if (listener == null) {
+ throw new AsterixException("Feed Listener is not registered");
+ }
+
+ boolean isFeedJointAvailable = listener.isFeedJointAvailable(feedJointKey);
if (!isFeedJointAvailable) {
- sourceFeedJoint = FeedLifecycleListener.INSTANCE.getAvailableFeedJoint(feedJointKey);
+ sourceFeedJoint = listener.getAvailableFeedJoint(feedJointKey);
if (sourceFeedJoint == null) { // the feed is currently not being ingested, i.e., it is unavailable.
connectionLocation = FeedRuntimeType.INTAKE;
- FeedId sourceFeedId = feedJointKey.getFeedId(); // the root/primary feedId
- Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, sourceFeedId.getFeedName());
+ 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>());
sourceFeedJoint = new FeedJoint(intakeFeedJointKey, primaryFeed.getFeedId(), connectionLocation,
FeedJointType.INTAKE, connectionId);
@@ -2306,7 +2287,7 @@
jointsToRegister.add(computeFeedJoint);
}
} else {
- sourceFeedJoint = FeedLifecycleListener.INSTANCE.getFeedJoint(feedJointKey);
+ sourceFeedJoint = listener.getFeedJoint(feedJointKey);
connectionLocation = sourceFeedJoint.getConnectionLocation();
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Feed joint " + sourceFeedJoint + " is available! need not apply any further computation");
@@ -2331,8 +2312,8 @@
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;
}
@@ -2354,16 +2335,16 @@
FeedMetadataUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(), mdTxnCtx);
Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
-
+ EntityId entityId = new EntityId(Feed.EXTENSION_NAME, feed.getDataverseName(), feed.getFeedName());
FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), cfs.getDatasetName().getValue());
IFeedLifecycleEventSubscriber eventSubscriber = new FeedLifecycleEventSubscriber();
- boolean isFeedConnectionActive = FeedLifecycleListener.INSTANCE.isFeedConnectionActive(connectionId,
- eventSubscriber);
- if (!isFeedConnectionActive) {
- throw new AsterixException("Feed " + feed.getFeedId().getFeedName() + " is currently not connected to "
+ 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!");
}
-
+ listener.registerFeedEventSubscriber(eventSubscriber);
MetadataLockManager.INSTANCE.disconnectFeedBegin(dataverseName, dataverseName + "." + datasetName,
dataverseName + "." + cfs.getFeedName());
try {
@@ -2373,8 +2354,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;
@@ -2405,8 +2386,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,
@@ -2418,14 +2399,28 @@
String dataverse = feedConnectionId.getFeedId().getDataverse();
String dataset = feedConnectionId.getDatasetName();
MetadataLockManager.INSTANCE.subscribeFeedBegin(dataverse, dataverse + "." + dataset,
- dataverse + "." + feedConnectionId.getFeedId().getFeedName());
+ dataverse + "." + feedConnectionId.getFeedId().getEntityName());
try {
JobSpecification alteredJobSpec = FeedMetadataUtil.alterJobSpecificationForFeed(compiled, feedConnectionId,
bfs.getSubscriptionRequest().getPolicyParameters());
+ FeedPolicyEntity policy = metadataProvider.findFeedPolicy(dataverse, bfs.getPolicy());
+ if (policy == null) {
+ policy = BuiltinFeedPolicies.getFeedPolicy(bfs.getPolicy());
+ if (policy == null) {
+ throw new AlgebricksException("Unknown feed policy:" + bfs.getPolicy());
+ }
+ }
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
if (compiled != null) {
+ FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
+ .getActiveEntityListener(bfs.getSubscriptionRequest().getReceivingFeedId());
+ 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);
JobUtils.runJob(hcc, alteredJobSpec, false);
}
@@ -2437,7 +2432,7 @@
throw e;
} finally {
MetadataLockManager.INSTANCE.subscribeFeedEnd(dataverse, dataverse + "." + dataset,
- dataverse + "." + feedConnectionId.getFeedId().getFeedName());
+ dataverse + "." + feedConnectionId.getFeedId().getEntityName());
}
}
@@ -2462,10 +2457,9 @@
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);
@@ -2473,8 +2467,8 @@
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);
@@ -2959,8 +2953,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);
+ DropStatement dropStmt =
+ new DropStatement(new Identifier(dataverseNameTo), pregelixStmt.getDatasetNameTo(), true);
this.handleDatasetDropStatement(metadataProvider, dropStmt, hcc);
IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(),
toIndex.getKeyFieldSourceIndicators(), false, null, toDataset.getDatasetDetails().isTemp());
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 06ec6bc..324194d 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
@@ -34,8 +34,8 @@
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.external.ActiveLifecycleListener;
import org.apache.asterix.app.external.ExternalLibraryUtils;
-import org.apache.asterix.app.external.FeedLifecycleListener;
import org.apache.asterix.common.api.AsterixThreadFactory;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
import org.apache.asterix.common.config.AsterixExternalProperties;
@@ -101,7 +101,7 @@
MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
AsterixAppContextInfo.getInstance().getCCApplicationContext()
- .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
+ .addJobLifecycleListener(ActiveLifecycleListener.INSTANCE);
AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
setupWebServer(externalProperties);
@@ -120,6 +120,7 @@
@Override
public void stop() throws Exception {
+ ActiveLifecycleListener.INSTANCE.stop();
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Stopping Asterix cluster controller");
}
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 9dd4025..138b620 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,8 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.app.external.FeedLifecycleListener;
+import org.apache.asterix.active.ActivePartitionMessage;
+import org.apache.asterix.app.external.ActiveLifecycleListener;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
@@ -37,7 +38,6 @@
import org.apache.asterix.common.messaging.TakeoverPartitionsResponseMessage;
import org.apache.asterix.common.messaging.api.IApplicationMessage;
import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.external.feed.message.FeedPartitionStartMessage;
import org.apache.asterix.om.util.AsterixClusterProperties;
import org.apache.hyracks.api.messages.IMessage;
import org.apache.hyracks.api.util.JavaSerializationUtils;
@@ -81,8 +81,8 @@
case COMPLETE_FAILBACK_RESPONSE:
handleCompleteFailbcakResponse(message);
break;
- case FEED_PROVIDER_READY:
- handleFeedProviderReady(message);
+ case ACTIVE_ENTITY_MESSAGE:
+ handleActiveEntityMessage(message);
break;
default:
LOGGER.warning("Unknown message: " + absMessage.getMessageType());
@@ -90,9 +90,8 @@
}
}
- private void handleFeedProviderReady(IMessage message) {
- FeedPartitionStartMessage msg = (FeedPartitionStartMessage) message;
- FeedLifecycleListener.INSTANCE.notifyPartitionStart(msg.getFeedId(), msg.getJobId());
+ private void handleActiveEntityMessage(IMessage message) {
+ ActiveLifecycleListener.INSTANCE.receive((ActivePartitionMessage) message);
}
private synchronized void handleResourceIdRequest(IMessage message, String nodeId) throws Exception {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 0989545..6bddfa5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -36,7 +36,6 @@
import org.apache.asterix.lang.common.base.IQueryRewriter;
import org.apache.asterix.lang.common.base.IRewriterFactory;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.base.Statement.Kind;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.FunctionDecl;
@@ -129,7 +128,7 @@
when(aqlMetadataProvider.findDataset(anyString(), anyString())).thenReturn(mock(Dataset.class));
for (Statement st : statements) {
- if (st.getKind() == Kind.QUERY) {
+ if (st.getKind() == Statement.QUERY) {
Query query = (Query) st;
IQueryRewriter rewriter = sqlppRewriterFactory.createQueryRewriter();
rewrite(rewriter, functions, query, aqlMetadataProvider,
@@ -157,7 +156,7 @@
private List<FunctionDecl> getDeclaredFunctions(List<Statement> statements) {
List<FunctionDecl> functionDecls = new ArrayList<FunctionDecl>();
for (Statement st : statements) {
- if (st.getKind().equals(Statement.Kind.FUNCTION_DECL)) {
+ if (st.getKind() == Statement.FUNCTION_DECL) {
functionDecls.add((FunctionDecl) st);
}
}
@@ -167,7 +166,7 @@
// Gets the default dataverse for the input statements.
private String getDefaultDataverse(List<Statement> statements) {
for (Statement st : statements) {
- if (st.getKind().equals(Statement.Kind.DATAVERSE_DECL)) {
+ if (st.getKind() == Statement.DATAVERSE_DECL) {
DataverseDecl dv = (DataverseDecl) st;
return dv.getDataverseName().getValue();
}
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 fba74e8..7940680 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,7 @@
COMPLETE_FAILBACK_REQUEST,
COMPLETE_FAILBACK_RESPONSE,
REPLICA_EVENT,
- FEED_PROVIDER_READY
+ ACTIVE_ENTITY_MESSAGE
}
public abstract ApplicationMessageType getMessageType();
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index e35146c..08a30e3 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -211,6 +211,21 @@
<artifactId>algebricks-compiler</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-active</artifactId>
+ <version>0.8.9-SNAPSHOT</version>
+ </dependency>
+ <dependency>
<groupId>com.kenai.nbpwr</groupId>
<artifactId>org-apache-commons-io</artifactId>
<version>1.3.1-201002241208</version>
@@ -255,21 +270,11 @@
<version>1.0.1-modified-01</version>
</dependency>
<dependency>
- <groupId>org.apache.hyracks</groupId>
- <artifactId>hyracks-hdfs-core</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
<groupId>jdom</groupId>
<artifactId>jdom</artifactId>
<version>1.0</version>
</dependency>
<dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-common</artifactId>
- <version>0.8.9-SNAPSHOT</version>
- </dependency>
- <dependency>
<groupId>com.microsoft.windowsazure</groupId>
<artifactId>microsoft-windowsazure-api</artifactId>
<version>0.4.4</version>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java
index 0dd87d6..dac8fbb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java
@@ -18,10 +18,10 @@
*/
package org.apache.asterix.external.feed.api;
-import org.apache.asterix.external.feed.watch.FeedJobInfo;
+import org.apache.asterix.active.ActiveJob;
public class FeedOperationCounter {
- private FeedJobInfo feedJobInfo;
+ private ActiveJob feedJobInfo;
private int partitionCount;
private boolean failedIngestion = false;
@@ -45,11 +45,15 @@
this.failedIngestion = failedIngestion;
}
- public FeedJobInfo getFeedJobInfo() {
+ public ActiveJob getFeedJobInfo() {
return feedJobInfo;
}
- public void setFeedJobInfo(FeedJobInfo feedJobInfo) {
+ public void setFeedJobInfo(ActiveJob feedJobInfo) {
this.feedJobInfo = feedJobInfo;
}
+
+ public int decrementAndGet() {
+ return --partitionCount;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
index 6865522..ce0b3d6 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
@@ -21,8 +21,8 @@
import java.io.Serializable;
import java.util.Map;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.external.feed.management.FeedId;
public interface IFeed extends Serializable {
@@ -46,7 +46,7 @@
public String getDataverseName();
- public FeedId getFeedId();
+ public EntityId getFeedId();
public Map<String, String> getAdapterConfiguration();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java
deleted file mode 100644
index 503715b..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.api;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedRuntimeManager;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-/**
- * Handle (de)registration of feeds for delivery of control messages.
- */
-public interface IFeedConnectionManager {
-
- /**
- * Allows registration of a feedRuntime.
- *
- * @param feedRuntime
- * @throws Exception
- */
- public void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime);
-
- /**
- * Obtain feed runtime corresponding to a feedRuntimeId
- *
- * @param feedRuntimeId
- * @return
- */
- public FeedRuntime getFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId);
-
- /**
- * De-register a feed
- *
- * @param feedConnection
- * @throws IOException
- */
- void deregisterFeed(FeedConnectionId feedConnection);
-
- /**
- * Obtain the feed runtime manager associated with a feed.
- *
- * @param feedConnection
- * @return
- */
- public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedConnection);
-
- /**
- * Allows de-registration of a feed runtime.
- *
- * @param feedRuntimeId
- */
- void deRegisterFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId);
-
- public List<FeedRuntimeId> getRegisteredRuntimes();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java
index 477ec36..548be4a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java
@@ -18,15 +18,16 @@
*/
package org.apache.asterix.external.feed.api;
+import java.io.Serializable;
import java.util.List;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.management.FeedJointKey;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-public interface IFeedJoint {
+public interface IFeedJoint extends Serializable {
public enum FeedJointType {
/** Feed Joint is located at the intake stage of a primary feed **/
@@ -100,7 +101,7 @@
*/
public void removeReceiver(FeedConnectionId connectionId);
- public FeedId getOwnerFeedId();
+ public EntityId getOwnerFeedId();
/**
* Add a feed connectionId to the set of registered subscribers
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.java
deleted file mode 100644
index 3302856..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.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.external.feed.api;
-
-import java.util.List;
-
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.hyracks.api.job.IJobLifecycleListener;
-
-public interface IFeedLifecycleListener extends IJobLifecycleListener {
- public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJoinKey);
-
- public boolean isFeedJointAvailable(FeedJointKey feedJoinKey);
-
- public List<FeedConnectionId> getActiveFeedConnections(FeedId feedId);
-
- public List<String> getComputeLocations(FeedId feedId);
-
- public List<String> getIntakeLocations(FeedId feedId);
-
- public List<String> getStoreLocations(FeedConnectionId feedId);
-
- public void registerFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber);
-
- public void deregisterFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber);
-
- public List<String> getCollectLocations(FeedConnectionId feedConnectionId);
-
- boolean isFeedConnectionActive(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber eventSubscriber);
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java
deleted file mode 100644
index b8375e3..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-public interface IFeedRuntime {
-
- public enum FeedRuntimeType {
- INTAKE,
- COLLECT,
- COMPUTE_COLLECT,
- COMPUTE,
- STORE,
- OTHER,
- ETS,
- JOIN
- }
-
- public enum Mode {
- PROCESS, // There is memory
- SPILL, // Memory budget has been consumed. Now we're writing to disk
- DISCARD // Memory budget has been consumed. Disk space budget has been consumed. Now we're
- // discarding
- }
-
- /**
- * @return the unique runtime id associated with the feedRuntime
- */
- public FeedRuntimeId getRuntimeId();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
index 1ca46ce..397f797 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
@@ -18,13 +18,14 @@
*/
package org.apache.asterix.external.feed.api;
+import org.apache.asterix.active.IActiveRuntime;
import org.apache.asterix.external.feed.runtime.CollectionRuntime;
import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* Represent a feed runtime whose output can be routed along other parallel path(s).
*/
-public interface ISubscribableRuntime extends IFeedRuntime {
+public interface ISubscribableRuntime extends IActiveRuntime {
/**
* @param collectionRuntime
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
index f356899..ae2e0b9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
@@ -21,13 +21,11 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
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.common.comm.io.FrameTupleAccessor;
/**
* Provides mechanism for distributing the frames, as received from an operator to a
@@ -38,7 +36,7 @@
public class DistributeFeedFrameWriter implements IFrameWriter {
/** A unique identifier for the feed to which the incoming tuples belong. **/
- private final FeedId feedId;
+ private final EntityId feedId;
/**
* An instance of FrameDistributor that provides the mechanism for distributing a frame to multiple readers, each
@@ -55,8 +53,8 @@
/** The value of the partition 'i' if this is the i'th instance of the associated operator **/
private final int partition;
- public DistributeFeedFrameWriter(IHyracksTaskContext ctx, FeedId feedId, IFrameWriter writer,
- FeedRuntimeType feedRuntimeType, int partition, FrameTupleAccessor fta) throws IOException {
+ public DistributeFeedFrameWriter(EntityId feedId, IFrameWriter writer, FeedRuntimeType feedRuntimeType,
+ int partition) throws IOException {
this.feedId = feedId;
this.frameDistributor = new FrameDistributor();
this.feedRuntimeType = feedRuntimeType;
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 8ed2bf9..2e1c83f 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
@@ -23,11 +23,12 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.management.ConcurrentFramePool;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.ConcurrentFramePool;
+import org.apache.asterix.active.FrameAction;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
+import org.apache.asterix.external.util.FeedUtils.Mode;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -67,7 +68,7 @@
private int numProcessedInMemory = 0;
private int numStalled = 0;
- public FeedRuntimeInputHandler(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ public FeedRuntimeInputHandler(IHyracksTaskContext ctx, FeedConnectionId connectionId, ActiveRuntimeId runtimeId,
IFrameWriter writer, FeedPolicyAccessor fpa, FrameTupleAccessor fta, ConcurrentFramePool framePool)
throws HyracksDataException {
this.writer = writer;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedCollectInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedCollectInfo.java
deleted file mode 100644
index 9f861d4..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedCollectInfo.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.management;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedCollectInfo extends FeedInfo {
- public FeedId sourceFeedId;
- public FeedConnectionId feedConnectionId;
- public List<String> collectLocations = new ArrayList<String>();
- public List<String> computeLocations = new ArrayList<String>();
- public List<String> storageLocations = new ArrayList<String>();
- public Map<String, String> feedPolicy;
- public String superFeedManagerHost;
- public int superFeedManagerPort;
- public boolean fullyConnected;
-
- public FeedCollectInfo(FeedId sourceFeedId, FeedConnectionId feedConnectionId, JobSpecification jobSpec,
- JobId jobId, Map<String, String> feedPolicy) {
- super(jobSpec, jobId, FeedInfoType.COLLECT);
- this.sourceFeedId = sourceFeedId;
- this.feedConnectionId = feedConnectionId;
- this.feedPolicy = feedPolicy;
- this.fullyConnected = true;
- }
-
- @Override
- public String toString() {
- return FeedInfoType.COLLECT + "[" + feedConnectionId + "]";
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
index 13f19b8..e2ab823 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
@@ -20,6 +20,8 @@
import java.io.Serializable;
+import org.apache.asterix.active.EntityId;
+
/**
* A unique identifier for a feed connection. A feed connection is an instance of a data feed that is flowing into a
* dataset.
@@ -27,22 +29,23 @@
public class FeedConnectionId implements Serializable {
private static final long serialVersionUID = 1L;
+ public static final String FEED_EXTENSION_NAME = "Feed";
- private final FeedId feedId; // Dataverse - Feed
- private final String datasetName; // Dataset <Dataset is empty in case of no target dataset>
+ private final EntityId feedId; // Dataverse - Feed
+ private final String datasetName; // Dataset <Dataset is empty in case of no target dataset>
private final int hash;
- public FeedConnectionId(FeedId feedId, String datasetName) {
+ public FeedConnectionId(EntityId feedId, String datasetName) {
this.feedId = feedId;
this.datasetName = datasetName;
this.hash = toString().hashCode();
}
public FeedConnectionId(String dataverse, String feedName, String datasetName) {
- this(new FeedId(dataverse, feedName), datasetName);
+ this(new EntityId(FEED_EXTENSION_NAME, dataverse, feedName), datasetName);
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java
deleted file mode 100644
index a746ef4..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.management;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedConnectionManager;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-/**
- * An implementation of the IFeedManager interface.
- * Provider necessary central repository for registering/retrieving
- * artifacts/services associated with a feed.
- */
-public class FeedConnectionManager implements IFeedConnectionManager {
-
- private static final Logger LOGGER = Logger.getLogger(FeedConnectionManager.class.getName());
-
- private Map<FeedConnectionId, FeedRuntimeManager> feedRuntimeManagers = new HashMap<FeedConnectionId, FeedRuntimeManager>();
- private final String nodeId;
-
- public FeedConnectionManager(String nodeId) {
- this.nodeId = nodeId;
- }
-
- @Override
- public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId) {
- return feedRuntimeManagers.get(feedId);
- }
-
- @Override
- public void deregisterFeed(FeedConnectionId feedId) {
- try {
- FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
- if (mgr != null) {
- mgr.close();
- feedRuntimeManagers.remove(feedId);
- }
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Exception in closing feed runtime" + e.getMessage());
- }
- }
-
- }
-
- @Override
- public synchronized void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime) {
- FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
- if (runtimeMgr == null) {
- runtimeMgr = new FeedRuntimeManager(connectionId, this);
- feedRuntimeManagers.put(connectionId, runtimeMgr);
- }
- runtimeMgr.registerFeedRuntime(feedRuntime.getRuntimeId(), feedRuntime);
- }
-
- @Override
- public void deRegisterFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId) {
- FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
- if (runtimeMgr != null) {
- runtimeMgr.deregisterFeedRuntime(feedRuntimeId);
- }
- }
-
- @Override
- public FeedRuntime getFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId) {
- FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
- return runtimeMgr != null ? runtimeMgr.getFeedRuntime(feedRuntimeId) : null;
- }
-
- @Override
- public String toString() {
- return "FeedManager " + "[" + nodeId + "]";
- }
-
- @Override
- public List<FeedRuntimeId> getRegisteredRuntimes() {
- List<FeedRuntimeId> runtimes = new ArrayList<FeedRuntimeId>();
- for (Entry<FeedConnectionId, FeedRuntimeManager> entry : feedRuntimeManagers.entrySet()) {
- runtimes.addAll(entry.getValue().getFeedRuntimes());
- }
- return runtimes;
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
index b1d3300..1106160 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
@@ -18,16 +18,20 @@
*/
package org.apache.asterix.external.feed.management;
+import java.io.Serializable;
import java.util.List;
import java.util.Map;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.commons.lang3.StringUtils;
/**
* A request for connecting a feed to a dataset.
*/
-public class FeedConnectionRequest {
+public class FeedConnectionRequest implements Serializable {
+
+ private static final long serialVersionUID = 1L;
public enum ConnectionStatus {
/** initial state upon creating a connection request **/
@@ -64,11 +68,11 @@
/** Target dataset associated with the connection request **/
private final String targetDataset;
- private final FeedId receivingFeedId;
+ private final EntityId receivingFeedId;
public FeedConnectionRequest(FeedJointKey feedPointKey, FeedRuntimeType connectionLocation,
List<String> functionsToApply, String targetDataset, String policy, Map<String, String> policyParameters,
- FeedId receivingFeedId) {
+ EntityId receivingFeedId) {
this.feedJointKey = feedPointKey;
this.connectionLocation = connectionLocation;
this.functionsToApply = functionsToApply;
@@ -103,7 +107,7 @@
return connectionLocation;
}
- public FeedId getReceivingFeedId() {
+ public EntityId getReceivingFeedId() {
return receivingFeedId;
}
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
new file mode 100644
index 0000000..40d2500
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
@@ -0,0 +1,644 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.feed.management;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.ActiveJob;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.external.feed.api.FeedOperationCounter;
+import org.apache.asterix.external.feed.api.IFeedJoint;
+import org.apache.asterix.external.feed.api.IFeedJoint.State;
+import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber;
+import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber.FeedLifecycleEvent;
+import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
+import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
+import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
+import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
+import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
+import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
+import org.apache.asterix.external.util.FeedUtils.JobType;
+import org.apache.asterix.om.util.AsterixAppContextInfo;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobInfo;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+import org.apache.log4j.Logger;
+
+public class FeedEventsListener implements IActiveEntityEventsListener {
+ private static final Logger LOGGER = Logger.getLogger(FeedEventsListener.class);
+ private final Map<EntityId, Pair<FeedOperationCounter, List<IFeedJoint>>> feedPipeline;
+ private final List<IFeedLifecycleEventSubscriber> subscribers;
+ private final Map<Long, ActiveJob> jobs;
+ private final Map<Long, ActiveJob> intakeJobs;
+ private final Map<EntityId, FeedIntakeInfo> entity2Intake;
+ private final Map<FeedConnectionId, FeedConnectJobInfo> connectJobInfos;
+ private EntityId entityId;
+ private IFeedJoint sourceFeedJoint;
+
+ public FeedEventsListener(EntityId entityId) {
+ this.entityId = entityId;
+ subscribers = new ArrayList<>();
+ jobs = new HashMap<>();
+ feedPipeline = new HashMap<>();
+ entity2Intake = new HashMap<>();
+ connectJobInfos = new HashMap<>();
+ intakeJobs = new HashMap<>();
+ }
+
+ @Override
+ public void notify(ActiveEvent event) {
+ try {
+ switch (event.getEventKind()) {
+ case JOB_START:
+ handleJobStartEvent(event);
+ break;
+ case JOB_FINISH:
+ handleJobFinishEvent(event);
+ break;
+ case PARTITION_EVENT:
+ handlePartitionStart(event);
+ break;
+ default:
+ LOGGER.warn("Unknown Feed Event" + event);
+ break;
+ }
+ } catch (Exception e) {
+ LOGGER.error("Unhandled Exception", e);
+ }
+ }
+
+ private synchronized void handleJobStartEvent(ActiveEvent message) throws Exception {
+ ActiveJob jobInfo = jobs.get(message.getJobId().getId());
+ JobType jobType = (JobType) jobInfo.getJobObject();
+ switch (jobType) {
+ case INTAKE:
+ handleIntakeJobStartMessage((FeedIntakeInfo) jobInfo);
+ break;
+ case FEED_CONNECT:
+ handleCollectJobStartMessage((FeedConnectJobInfo) jobInfo);
+ break;
+ default:
+ }
+ }
+
+ private synchronized void handleJobFinishEvent(ActiveEvent message) throws Exception {
+ ActiveJob jobInfo = jobs.get(message.getJobId().getId());
+ JobType jobType = (JobType) jobInfo.getJobObject();
+ switch (jobType) {
+ case FEED_CONNECT:
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Collect Job finished for " + jobInfo);
+ }
+ handleFeedCollectJobFinishMessage((FeedConnectJobInfo) jobInfo);
+ break;
+ case INTAKE:
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Intake Job finished for feed intake " + jobInfo.getJobId());
+ }
+ handleFeedIntakeJobFinishMessage((FeedIntakeInfo) jobInfo, message);
+ break;
+ default:
+ break;
+ }
+ }
+
+ private synchronized void handlePartitionStart(ActiveEvent message) {
+ ActiveJob jobInfo = jobs.get(message.getJobId().getId());
+ JobType jobType = (JobType) jobInfo.getJobObject();
+ switch (jobType) {
+ case FEED_CONNECT:
+ ((FeedConnectJobInfo) jobInfo).partitionStart();
+ if (((FeedConnectJobInfo) jobInfo).collectionStarted()) {
+ notifyFeedEventSubscribers(FeedLifecycleEvent.FEED_COLLECT_STARTED);
+ }
+ break;
+ case INTAKE:
+ handleIntakePartitionStarts(message, jobInfo);
+ break;
+ default:
+ break;
+
+ }
+ }
+
+ private void handleIntakePartitionStarts(ActiveEvent message, ActiveJob jobInfo) {
+ if (feedPipeline.get(message.getFeedId()).first.decrementAndGet() == 0) {
+ ((FeedIntakeInfo) jobInfo).getIntakeFeedJoint().setState(State.ACTIVE);
+ jobInfo.setState(ActivityState.ACTIVE);
+ notifyFeedEventSubscribers(FeedLifecycleEvent.FEED_INTAKE_STARTED);
+ }
+ }
+
+ public synchronized void registerFeedJoint(IFeedJoint feedJoint, int numOfPrividers) {
+ Pair<FeedOperationCounter, List<IFeedJoint>> feedJointsOnPipeline =
+ feedPipeline.get(feedJoint.getOwnerFeedId());
+ if (feedJointsOnPipeline == null) {
+ feedJointsOnPipeline = new Pair<>(new FeedOperationCounter(numOfPrividers), new ArrayList<IFeedJoint>());
+ feedPipeline.put(feedJoint.getOwnerFeedId(), feedJointsOnPipeline);
+ feedJointsOnPipeline.second.add(feedJoint);
+ } else {
+ if (!feedJointsOnPipeline.second.contains(feedJoint)) {
+ feedJointsOnPipeline.second.add(feedJoint);
+ } else {
+ throw new IllegalArgumentException("Feed joint " + feedJoint + " already registered");
+ }
+ }
+ }
+
+ public synchronized void deregisterFeedIntakeJob(JobId jobId) {
+ FeedIntakeInfo info = (FeedIntakeInfo) intakeJobs.remove(jobId.getId());
+ jobs.remove(jobId.getId());
+ entity2Intake.remove(info.getFeedId());
+ List<IFeedJoint> joints = feedPipeline.get(info.getFeedId()).second;
+ joints.remove(info.getIntakeFeedJoint());
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Deregistered feed intake job [" + jobId + "]");
+ }
+ }
+
+ private static synchronized void handleIntakeJobStartMessage(FeedIntakeInfo intakeJobInfo) throws Exception {
+ List<OperatorDescriptorId> intakeOperatorIds = new ArrayList<>();
+ Map<OperatorDescriptorId, IOperatorDescriptor> operators = intakeJobInfo.getSpec().getOperatorMap();
+ for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
+ IOperatorDescriptor opDesc = entry.getValue();
+ if (opDesc instanceof FeedIntakeOperatorDescriptor) {
+ intakeOperatorIds.add(opDesc.getOperatorId());
+ }
+ }
+
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(intakeJobInfo.getJobId());
+ List<String> intakeLocations = new ArrayList<>();
+ for (OperatorDescriptorId intakeOperatorId : intakeOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(intakeOperatorId);
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ intakeLocations.add(operatorLocations.get(i));
+ }
+ }
+ // intakeLocations is an ordered list;
+ // element at position i corresponds to location of i'th instance of operator
+ intakeJobInfo.setIntakeLocation(intakeLocations);
+ }
+
+ public IFeedJoint getSourceFeedJoint(FeedConnectionId connectionId) {
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ if (cInfo != null) {
+ return cInfo.getSourceFeedJoint();
+ }
+ return null;
+ }
+
+ public synchronized void registerFeedIntakeJob(EntityId feedId, JobId jobId, JobSpecification jobSpec)
+ throws HyracksDataException {
+ if (entity2Intake.get(feedId) != null) {
+ throw new IllegalStateException("Feed already has an intake job");
+ }
+ if (intakeJobs.get(jobId.getId()) != null) {
+ throw new IllegalStateException("Feed job already registered in intake jobs");
+ }
+ if (jobs.get(jobId.getId()) != null) {
+ throw new IllegalStateException("Feed job already registered in all jobs");
+ }
+
+ Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(feedId);
+ sourceFeedJoint = null;
+ for (IFeedJoint joint : pair.second) {
+ if (joint.getType().equals(IFeedJoint.FeedJointType.INTAKE)) {
+ sourceFeedJoint = joint;
+ break;
+ }
+ }
+
+ if (sourceFeedJoint != null) {
+ FeedIntakeInfo intakeJobInfo =
+ new FeedIntakeInfo(jobId, ActivityState.CREATED, feedId, sourceFeedJoint, jobSpec);
+ pair.first.setFeedJobInfo(intakeJobInfo);
+ entity2Intake.put(feedId, intakeJobInfo);
+ jobs.put(jobId.getId(), intakeJobInfo);
+ intakeJobs.put(jobId.getId(), intakeJobInfo);
+
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Registered feed intake [" + jobId + "]" + " for feed " + feedId);
+ }
+ } else {
+ throw new HyracksDataException(
+ "Could not register feed intake job [" + jobId + "]" + " for feed " + feedId);
+ }
+ }
+
+ public synchronized void registerFeedCollectionJob(EntityId sourceFeedId, FeedConnectionId connectionId,
+ JobId jobId, JobSpecification jobSpec, Map<String, String> feedPolicy) {
+ if (jobs.get(jobId.getId()) != null) {
+ throw new IllegalStateException("Feed job already registered");
+ }
+ if (connectJobInfos.containsKey(jobId.getId())) {
+ throw new IllegalStateException("Feed job already registered");
+ }
+
+ List<IFeedJoint> feedJoints = feedPipeline.get(sourceFeedId).second;
+ FeedConnectionId cid = null;
+ IFeedJoint collectionSourceFeedJoint = null;
+ for (IFeedJoint joint : feedJoints) {
+ cid = joint.getReceiver(connectionId);
+ if (cid != null) {
+ collectionSourceFeedJoint = joint;
+ break;
+ }
+ }
+
+ if (cid != null) {
+ FeedConnectJobInfo cInfo = new FeedConnectJobInfo(sourceFeedId, jobId, ActivityState.CREATED, connectionId,
+ collectionSourceFeedJoint, null, jobSpec, feedPolicy);
+ jobs.put(jobId.getId(), cInfo);
+ connectJobInfos.put(connectionId, cInfo);
+
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Registered feed connection [" + jobId + "]" + " for feed " + connectionId);
+ }
+ } else {
+ LOGGER.warn(
+ "Could not register feed collection job [" + jobId + "]" + " for feed connection " + connectionId);
+ }
+ }
+
+ @Override
+ public void notifyJobCreation(JobId jobId, JobSpecification spec) {
+ FeedConnectionId feedConnectionId = null;
+ Map<String, String> feedPolicy = null;
+ try {
+ for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
+ if (opDesc instanceof FeedCollectOperatorDescriptor) {
+ feedConnectionId = ((FeedCollectOperatorDescriptor) opDesc).getFeedConnectionId();
+ feedPolicy = ((FeedCollectOperatorDescriptor) opDesc).getFeedPolicyProperties();
+ registerFeedCollectionJob(((FeedCollectOperatorDescriptor) opDesc).getSourceFeedId(),
+ feedConnectionId, jobId, spec, feedPolicy);
+ return;
+ } else if (opDesc instanceof FeedIntakeOperatorDescriptor) {
+ registerFeedIntakeJob(((FeedIntakeOperatorDescriptor) opDesc).getFeedId(), jobId, spec);
+ return;
+ }
+ }
+ } catch (Exception e) {
+ LOGGER.error(e);
+ }
+ }
+
+ public synchronized List<String> getConnectionLocations(IFeedJoint feedJoint, final FeedConnectionRequest request)
+ throws Exception {
+ List<String> locations = null;
+ switch (feedJoint.getType()) {
+ case COMPUTE:
+ FeedConnectionId connectionId = feedJoint.getProvider();
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ locations = cInfo.getComputeLocations();
+ break;
+ case INTAKE:
+ FeedIntakeInfo intakeInfo = entity2Intake.get(feedJoint.getOwnerFeedId());
+ locations = intakeInfo.getIntakeLocation();
+ break;
+ default:
+ break;
+ }
+ return locations;
+ }
+
+ private void notifyFeedEventSubscribers(FeedLifecycleEvent event) {
+ if (subscribers != null && !subscribers.isEmpty()) {
+ for (IFeedLifecycleEventSubscriber subscriber : subscribers) {
+ subscriber.handleFeedEvent(event);
+ }
+ }
+ }
+
+ private synchronized void handleFeedIntakeJobFinishMessage(FeedIntakeInfo intakeInfo, ActiveEvent message)
+ throws Exception {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(message.getJobId());
+ JobStatus status = info.getStatus();
+ EntityId feedId = intakeInfo.getFeedId();
+ Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(feedId);
+ if (status.equals(JobStatus.FAILURE)) {
+ pair.first.setFailedIngestion(true);
+ }
+ // remove feed joints
+ deregisterFeedIntakeJob(message.getJobId());
+ // notify event listeners
+ feedPipeline.remove(feedId);
+ entity2Intake.remove(feedId);
+ notifyFeedEventSubscribers(pair.first.isFailedIngestion() ? FeedLifecycleEvent.FEED_INTAKE_FAILURE
+ : FeedLifecycleEvent.FEED_INTAKE_ENDED);
+ }
+
+ private synchronized void handleFeedCollectJobFinishMessage(FeedConnectJobInfo cInfo) throws Exception {
+ FeedConnectionId connectionId = cInfo.getConnectionId();
+
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(cInfo.getJobId());
+ JobStatus status = info.getStatus();
+ boolean failure = status != null && status.equals(JobStatus.FAILURE);
+ FeedPolicyAccessor fpa = new FeedPolicyAccessor(cInfo.getFeedPolicy());
+ boolean retainSubsription =
+ cInfo.getState().equals(ActivityState.UNDER_RECOVERY) || (failure && fpa.continueOnHardwareFailure());
+
+ if (!retainSubsription) {
+ IFeedJoint feedJoint = cInfo.getSourceFeedJoint();
+ feedJoint.removeReceiver(connectionId);
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info(
+ "Subscription " + cInfo.getConnectionId() + " completed successfully. Removed subscription");
+ }
+ }
+
+ connectJobInfos.remove(connectionId);
+ jobs.remove(cInfo.getJobId().getId());
+ // notify event listeners
+ FeedLifecycleEvent event =
+ failure ? FeedLifecycleEvent.FEED_COLLECT_FAILURE : FeedLifecycleEvent.FEED_COLLECT_ENDED;
+ notifyFeedEventSubscribers(event);
+ }
+
+ public List<String> getFeedStorageLocations(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getStorageLocations();
+ }
+
+ public List<String> getFeedCollectLocations(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getCollectLocations();
+ }
+
+ public List<String> getFeedIntakeLocations(EntityId feedId) {
+ return entity2Intake.get(feedId).getIntakeLocation();
+ }
+
+ public JobId getFeedCollectJobId(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getJobId();
+ }
+
+ public boolean isFeedPointAvailable(FeedJointKey feedJointKey) {
+ List<IFeedJoint> joints = feedPipeline.containsKey(feedJointKey.getFeedId())
+ ? feedPipeline.get(feedJointKey.getFeedId()).second : null;
+ if (joints != null && !joints.isEmpty()) {
+ for (IFeedJoint joint : joints) {
+ if (joint.getFeedJointKey().equals(feedJointKey)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ public Collection<IFeedJoint> getFeedIntakeJoints() {
+ List<IFeedJoint> intakeFeedPoints = new ArrayList<>();
+ for (FeedIntakeInfo info : entity2Intake.values()) {
+ intakeFeedPoints.add(info.getIntakeFeedJoint());
+ }
+ return intakeFeedPoints;
+ }
+
+ public IFeedJoint getFeedJoint(FeedJointKey feedPointKey) {
+ List<IFeedJoint> joints = feedPipeline.containsKey(feedPointKey.getFeedId())
+ ? feedPipeline.get(feedPointKey.getFeedId()).second : null;
+ if (joints != null && !joints.isEmpty()) {
+ for (IFeedJoint joint : joints) {
+ if (joint.getFeedJointKey().equals(feedPointKey)) {
+ return joint;
+ }
+ }
+ }
+ return null;
+ }
+
+ public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
+ IFeedJoint feedJoint = getFeedJoint(feedJointKey);
+ if (feedJoint != null) {
+ return feedJoint;
+ } else {
+ String jointKeyString = feedJointKey.getStringRep();
+ List<IFeedJoint> jointsOnPipeline = feedPipeline.containsKey(feedJointKey.getFeedId())
+ ? feedPipeline.get(feedJointKey.getFeedId()).second : null;
+ IFeedJoint candidateJoint = null;
+ if (jointsOnPipeline != null) {
+ for (IFeedJoint joint : jointsOnPipeline) {
+ if (jointKeyString.contains(joint.getFeedJointKey().getStringRep()) && (candidateJoint == null
+ || /*found feed point is a super set of the earlier find*/joint.getFeedJointKey()
+ .getStringRep().contains(candidateJoint.getFeedJointKey().getStringRep()))) {
+ candidateJoint = joint;
+ }
+ }
+ }
+ return candidateJoint;
+ }
+ }
+
+ public JobSpecification getCollectJobSpecification(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getSpec();
+ }
+
+ public IFeedJoint getFeedPoint(EntityId sourceFeedId, IFeedJoint.FeedJointType type) {
+ List<IFeedJoint> joints = feedPipeline.get(sourceFeedId).second;
+ for (IFeedJoint joint : joints) {
+ if (joint.getType().equals(type)) {
+ return joint;
+ }
+ }
+ return null;
+ }
+
+ private void setLocations(FeedConnectJobInfo cInfo) {
+ JobSpecification jobSpec = cInfo.getSpec();
+
+ List<OperatorDescriptorId> collectOperatorIds = new ArrayList<>();
+ List<OperatorDescriptorId> computeOperatorIds = new ArrayList<>();
+ List<OperatorDescriptorId> storageOperatorIds = new ArrayList<>();
+
+ Map<OperatorDescriptorId, IOperatorDescriptor> operators = jobSpec.getOperatorMap();
+ for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
+ IOperatorDescriptor opDesc = entry.getValue();
+ IOperatorDescriptor actualOp;
+ if (opDesc instanceof FeedMetaOperatorDescriptor) {
+ actualOp = ((FeedMetaOperatorDescriptor) opDesc).getCoreOperator();
+ } else {
+ actualOp = opDesc;
+ }
+
+ if (actualOp instanceof AlgebricksMetaOperatorDescriptor) {
+ AlgebricksMetaOperatorDescriptor op = (AlgebricksMetaOperatorDescriptor) actualOp;
+ IPushRuntimeFactory[] runtimeFactories = op.getPipeline().getRuntimeFactories();
+ boolean computeOp = false;
+ for (IPushRuntimeFactory rf : runtimeFactories) {
+ if (rf instanceof AssignRuntimeFactory) {
+ IConnectorDescriptor connDesc = jobSpec.getOperatorInputMap().get(op.getOperatorId()).get(0);
+ IOperatorDescriptor sourceOp =
+ jobSpec.getConnectorOperatorMap().get(connDesc.getConnectorId()).getLeft().getLeft();
+ if (sourceOp instanceof FeedCollectOperatorDescriptor) {
+ computeOp = true;
+ break;
+ }
+ }
+ }
+ if (computeOp) {
+ computeOperatorIds.add(entry.getKey());
+ }
+ } else if (actualOp instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
+ storageOperatorIds.add(entry.getKey());
+ } else if (actualOp instanceof FeedCollectOperatorDescriptor) {
+ collectOperatorIds.add(entry.getKey());
+ }
+ }
+
+ try {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(cInfo.getJobId());
+ List<String> collectLocations = new ArrayList<>();
+ for (OperatorDescriptorId collectOpId : collectOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(collectOpId);
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ collectLocations.add(operatorLocations.get(i));
+ }
+ }
+
+ List<String> computeLocations = new ArrayList<>();
+ for (OperatorDescriptorId computeOpId : computeOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(computeOpId);
+ if (operatorLocations != null) {
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ computeLocations.add(operatorLocations.get(i));
+ }
+ } else {
+ computeLocations.clear();
+ computeLocations.addAll(collectLocations);
+ }
+ }
+
+ List<String> storageLocations = new ArrayList<>();
+ for (OperatorDescriptorId storageOpId : storageOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(storageOpId);
+ if (operatorLocations == null) {
+ continue;
+ }
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ storageLocations.add(operatorLocations.get(i));
+ }
+ }
+ cInfo.setCollectLocations(collectLocations);
+ cInfo.setComputeLocations(computeLocations);
+ cInfo.setStorageLocations(storageLocations);
+
+ } catch (Exception e) {
+ LOGGER.error("Error while setting feed active locations", e);
+ }
+
+ }
+
+ public synchronized void registerFeedEventSubscriber(IFeedLifecycleEventSubscriber subscriber) {
+ subscribers.add(subscriber);
+ }
+
+ public void deregisterFeedEventSubscriber(IFeedLifecycleEventSubscriber subscriber) {
+ subscribers.remove(subscriber);
+ }
+
+ public synchronized boolean isFeedConnectionActive(FeedConnectionId connectionId,
+ IFeedLifecycleEventSubscriber eventSubscriber) {
+ boolean active = false;
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ if (cInfo != null) {
+ active = cInfo.getState().equals(ActivityState.ACTIVE);
+ }
+ if (active) {
+ registerFeedEventSubscriber(eventSubscriber);
+ }
+ return active;
+ }
+
+ public FeedConnectJobInfo getFeedConnectJobInfo(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId);
+ }
+
+ private void handleCollectJobStartMessage(FeedConnectJobInfo cInfo) throws ACIDException {
+ // set locations of feed sub-operations (intake, compute, store)
+ setLocations(cInfo);
+ Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(cInfo.getConnectionId().getFeedId());
+ // activate joints
+ List<IFeedJoint> joints = pair.second;
+ for (IFeedJoint joint : joints) {
+ if (joint.getProvider().equals(cInfo.getConnectionId())) {
+ joint.setState(State.ACTIVE);
+ if (joint.getType().equals(IFeedJoint.FeedJointType.COMPUTE)) {
+ cInfo.setComputeFeedJoint(joint);
+ }
+ }
+ }
+ cInfo.setState(ActivityState.ACTIVE);
+ }
+
+ public synchronized boolean isConnectedToDataset(String datasetName) {
+ for (FeedConnectionId connection : connectJobInfos.keySet()) {
+ if (connection.getDatasetName().equals(datasetName)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public FeedConnectionId[] getConnections() {
+ return connectJobInfos.keySet().toArray(new FeedConnectionId[connectJobInfos.size()]);
+ }
+
+ public boolean isFeedJointAvailable(FeedJointKey feedJointKey) {
+ return isFeedPointAvailable(feedJointKey);
+ }
+
+ @Override
+ public boolean isEntityActive() {
+ return !jobs.isEmpty();
+ }
+
+ @Override
+ public EntityId getEntityId() {
+ return entityId;
+ }
+
+ public IFeedJoint getSourceFeedJoint() {
+ return sourceFeedJoint;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedId.java
deleted file mode 100644
index 3145d72..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedId.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.management;
-
-import java.io.Serializable;
-
-/**
- * A unique identifier for a data feed.
- */
-public class FeedId implements Serializable {
-
- private static final long serialVersionUID = 1L;
-
- private final String dataverse;
- private final String feedName;
-
- public FeedId(String dataverse, String feedName) {
- this.dataverse = dataverse;
- this.feedName = feedName;
- }
-
- public String getDataverse() {
- return dataverse;
- }
-
- public String getFeedName() {
- return feedName;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o == null || !(o instanceof FeedId)) {
- return false;
- }
- if (this == o || ((FeedId) o).getFeedName().equals(feedName) && ((FeedId) o).getDataverse().equals(dataverse)) {
- return true;
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return toString().hashCode();
- }
-
- @Override
- public String toString() {
- return dataverse + "." + feedName;
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java
index edbaf7c..2905bb2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java
@@ -18,8 +18,10 @@
*/
package org.apache.asterix.external.feed.management;
+import java.io.Serializable;
import java.util.List;
+import org.apache.asterix.active.EntityId;
import org.apache.commons.lang3.StringUtils;
/**
@@ -28,13 +30,14 @@
* constitute the feed. The feed joint acts as a network tap and allows the flowing data to be
* routed to multiple paths.
*/
-public class FeedJointKey {
+public class FeedJointKey implements Serializable {
- private final FeedId primaryFeedId;
+ private static final long serialVersionUID = 1L;
+ private final EntityId primaryFeedId;
private final List<String> appliedFunctions;
private final String stringRep;
- public FeedJointKey(FeedId feedId, List<String> appliedFunctions) {
+ public FeedJointKey(EntityId feedId, List<String> appliedFunctions) {
this.primaryFeedId = feedId;
this.appliedFunctions = appliedFunctions;
StringBuilder builder = new StringBuilder();
@@ -44,7 +47,7 @@
stringRep = builder.toString();
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return primaryFeedId;
}
@@ -68,8 +71,9 @@
@Override
public boolean equals(Object o) {
- if (this == o)
+ if (this == o) {
return true;
+ }
if (o == null || !(o instanceof FeedJointKey)) {
return false;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java
deleted file mode 100644
index 62f4c6c..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.management;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.asterix.common.config.AsterixFeedProperties;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.feed.api.IFeedConnectionManager;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * An implementation of the IFeedManager interface.
- * Provider necessary central repository for registering/retrieving
- * artifacts/services associated with a feed.
- */
-public class FeedManager {
-
- private final Map<FeedRuntimeId, ISubscribableRuntime> subscribableRuntimes;
-
- private final IFeedConnectionManager feedConnectionManager;
-
- private final ConcurrentFramePool feedMemoryManager;
-
- private final AsterixFeedProperties asterixFeedProperties;
-
- private final String nodeId;
-
- private final int frameSize;
-
- public FeedManager(String nodeId, AsterixFeedProperties feedProperties, int frameSize)
- throws AsterixException, HyracksDataException {
- this.nodeId = nodeId;
- this.feedConnectionManager = new FeedConnectionManager(nodeId);
- this.feedMemoryManager =
- new ConcurrentFramePool(nodeId, feedProperties.getMemoryComponentGlobalBudget(), frameSize);
- this.frameSize = frameSize;
- this.asterixFeedProperties = feedProperties;
- this.subscribableRuntimes = new ConcurrentHashMap<FeedRuntimeId, ISubscribableRuntime>();
- }
-
- public IFeedConnectionManager getFeedConnectionManager() {
- return feedConnectionManager;
- }
-
- public ConcurrentFramePool getFeedMemoryManager() {
- return feedMemoryManager;
- }
-
- public int getFrameSize() {
- return frameSize;
- }
-
- public void registerFeedSubscribableRuntime(ISubscribableRuntime subscribableRuntime) {
- FeedRuntimeId sid = subscribableRuntime.getRuntimeId();
- if (!subscribableRuntimes.containsKey(sid)) {
- subscribableRuntimes.put(sid, subscribableRuntime);
- }
- }
-
- public void deregisterFeedSubscribableRuntime(FeedRuntimeId subscribableRuntimeId) {
- subscribableRuntimes.remove(subscribableRuntimeId);
- }
-
- public ISubscribableRuntime getSubscribableRuntime(FeedRuntimeId subscribableRuntimeId) {
- return subscribableRuntimes.get(subscribableRuntimeId);
- }
-
- @Override
- public String toString() {
- return "FeedManager " + "[" + nodeId + "]";
- }
-
- public AsterixFeedProperties getAsterixFeedProperties() {
- return asterixFeedProperties;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedRuntimeManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedRuntimeManager.java
deleted file mode 100644
index f7e98f7..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedRuntimeManager.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.management;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedConnectionManager;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-public class FeedRuntimeManager {
-
- private static Logger LOGGER = Logger.getLogger(FeedRuntimeManager.class.getName());
-
- private final FeedConnectionId connectionId;
- private final IFeedConnectionManager connectionManager;
- private final Map<FeedRuntimeId, FeedRuntime> feedRuntimes;
-
- private final ExecutorService executorService;
-
- public FeedRuntimeManager(FeedConnectionId connectionId, IFeedConnectionManager feedConnectionManager) {
- this.connectionId = connectionId;
- this.feedRuntimes = new ConcurrentHashMap<FeedRuntimeId, FeedRuntime>();
- this.executorService = Executors.newCachedThreadPool();
- this.connectionManager = feedConnectionManager;
- }
-
- public void close() throws IOException {
- if (executorService != null) {
- executorService.shutdownNow();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Shut down executor service for :" + connectionId);
- }
- }
- }
-
- public FeedRuntime getFeedRuntime(FeedRuntimeId runtimeId) {
- return feedRuntimes.get(runtimeId);
- }
-
- public void registerFeedRuntime(FeedRuntimeId runtimeId, FeedRuntime feedRuntime) {
- feedRuntimes.put(runtimeId, feedRuntime);
- }
-
- public synchronized void deregisterFeedRuntime(FeedRuntimeId runtimeId) {
- feedRuntimes.remove(runtimeId);
- if (feedRuntimes.isEmpty()) {
- connectionManager.deregisterFeed(connectionId);
- }
- }
-
- public ExecutorService getExecutorService() {
- return executorService;
- }
-
- public Set<FeedRuntimeId> getFeedRuntimes() {
- return feedRuntimes.keySet();
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java
index b0f7624..06aafdd 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java
@@ -18,22 +18,24 @@
*/
package org.apache.asterix.external.feed.message;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.util.FeedConstants;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.json.JSONException;
import org.json.JSONObject;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.util.FeedConstants;
/**
- * A feed control message indicating the need to end the feed. This message is dispatched
- * to all locations that host an operator involved in the feed pipeline.
+ * @deprecated A feed control message indicating the need to end the feed. This message is dispatched
+ * to all locations that host an operator involved in the feed pipeline.
+ * Instead, use IMessageBroker messages
*/
+@Deprecated
public class EndFeedMessage extends FeedMessage {
private static final long serialVersionUID = 1L;
- private final FeedId sourceFeedId;
+ private final EntityId sourceFeedId;
private final FeedConnectionId connectionId;
@@ -48,7 +50,7 @@
DISCONTINUE_SOURCE
}
- public EndFeedMessage(FeedConnectionId connectionId, FeedRuntimeType sourceRuntimeType, FeedId sourceFeedId,
+ public EndFeedMessage(FeedConnectionId connectionId, FeedRuntimeType sourceRuntimeType, EntityId sourceFeedId,
boolean completeDisconnection, EndMessageType endMessageType) {
super(MessageType.END);
this.connectionId = connectionId;
@@ -67,7 +69,7 @@
return sourceRuntimeType;
}
- public FeedId getSourceFeedId() {
+ public EntityId getSourceFeedId() {
return sourceFeedId;
}
@@ -84,7 +86,7 @@
JSONObject obj = new JSONObject();
obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getEntityName());
obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
return obj;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java
index f2b354b..4f57fb5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java
@@ -18,14 +18,14 @@
*/
package org.apache.asterix.external.feed.message;
-import org.apache.asterix.external.feed.api.IFeedMessage;
+import org.apache.asterix.active.IActiveMessage;
import org.apache.hyracks.api.dataflow.value.JSONSerializable;
/**
* A control message that can be sent to the runtime instance of a
* feed's adapter.
*/
-public abstract class FeedMessage implements IFeedMessage, JSONSerializable {
+public abstract class FeedMessage implements IActiveMessage, JSONSerializable {
private static final long serialVersionUID = 1L;
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 b604db5..f2f0747 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
@@ -22,9 +22,9 @@
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
+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.asterix.external.feed.management.FeedId;
import org.apache.log4j.Logger;
/**
@@ -34,7 +34,7 @@
private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
- private final FeedId feedId; // (dataverse-feed)
+ private final EntityId feedId; // (dataverse-feed)
private final FeedAdapter feedAdapter; // The adapter
@@ -49,7 +49,7 @@
private volatile boolean done = false;
private volatile boolean failed = false;
- public AdapterRuntimeManager(FeedId feedId, FeedAdapter feedAdapter, DistributeFeedFrameWriter writer,
+ public AdapterRuntimeManager(EntityId feedId, FeedAdapter feedAdapter, DistributeFeedFrameWriter writer,
int partition) {
this.feedId = feedId;
this.feedAdapter = feedAdapter;
@@ -86,7 +86,7 @@
}
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
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 6b1c5b8..294642e 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,6 +20,8 @@
import java.util.Map;
+import org.apache.asterix.active.ActiveRuntime;
+import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
@@ -32,16 +34,17 @@
* intake job. For a secondary feed, tuples are collected from the intake/compute
* runtime associated with the source feed.
*/
-public class CollectionRuntime extends FeedRuntime implements ISubscriberRuntime {
+public class CollectionRuntime extends ActiveRuntime implements ISubscriberRuntime {
- 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;
- public CollectionRuntime(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ISubscribableRuntime sourceRuntime,
- Map<String, String> feedPolicy, IHyracksTaskContext ctx, FeedFrameCollector frameCollector) {
+ public CollectionRuntime(FeedConnectionId connectionId, ActiveRuntimeId runtimeId,
+ ISubscribableRuntime sourceRuntime, Map<String, String> feedPolicy, IHyracksTaskContext ctx,
+ FeedFrameCollector frameCollector) {
super(runtimeId);
this.connectionId = connectionId;
this.sourceRuntime = sourceRuntime;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java
deleted file mode 100644
index 18d4cff..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.runtime;
-
-import java.io.Serializable;
-
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedId;
-
-public class FeedRuntimeId implements Serializable {
-
- private static final long serialVersionUID = 1L;
-
- public static final String DEFAULT_TARGET_ID = "N/A";
-
- private final FeedId feedId;
- private final FeedRuntimeType runtimeType;
- private final int partition;
- private final String targetId;
- private final int hashCode;
-
- public FeedRuntimeId(FeedId feedId, FeedRuntimeType runtimeType, int partition, String targetId) {
- this.feedId = feedId;
- this.runtimeType = runtimeType;
- this.partition = partition;
- this.targetId = targetId;
- this.hashCode = toString().hashCode();
- }
-
- @Override
- public String toString() {
- return runtimeType + "(" + feedId + ")" + "[" + partition + "]" + "==>" + "{" + targetId + "}";
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (!(o instanceof FeedRuntimeId)) {
- return false;
- }
- FeedRuntimeId other = (FeedRuntimeId) o;
- return (other.feedId.equals(feedId) && other.getFeedRuntimeType().equals(runtimeType)
- && other.getTargetId().equals(targetId) && other.getPartition() == partition);
- }
-
- @Override
- public int hashCode() {
- return hashCode;
- }
-
- public FeedRuntimeType getFeedRuntimeType() {
- return runtimeType;
- }
-
- public int getPartition() {
- return partition;
- }
-
- public FeedRuntimeType getRuntimeType() {
- return runtimeType;
- }
-
- public String getTargetId() {
- return targetId;
- }
-
- public FeedId getFeedId() {
- return feedId;
- }
-}
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 6cdc2af..8ee3e2b 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
@@ -20,10 +20,11 @@
import java.util.logging.Level;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -34,7 +35,7 @@
private final IHyracksTaskContext ctx;
private int numSubscribers = 0;
- public IngestionRuntime(FeedId feedId, FeedRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
+ public IngestionRuntime(EntityId feedId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
AdapterRuntimeManager adaptorRuntimeManager, IHyracksTaskContext ctx) {
super(feedId, runtimeId, feedWriter);
this.adapterRuntimeManager = adaptorRuntimeManager;
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 e060e27..423e599 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,26 +22,28 @@
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.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-import org.apache.asterix.external.feed.management.FeedId;
-public abstract class SubscribableRuntime extends FeedRuntime implements ISubscribableRuntime {
+public abstract class SubscribableRuntime extends ActiveRuntime implements ISubscribableRuntime {
protected static final Logger LOGGER = Logger.getLogger(SubscribableRuntime.class.getName());
- protected final FeedId feedId;
+ protected final EntityId feedId;
protected final List<ISubscriberRuntime> subscribers;
protected final DistributeFeedFrameWriter dWriter;
- public SubscribableRuntime(FeedId feedId, FeedRuntimeId runtimeId, DistributeFeedFrameWriter dWriter) {
+ public SubscribableRuntime(EntityId feedId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter dWriter) {
super(runtimeId);
this.feedId = feedId;
this.dWriter = dWriter;
this.subscribers = new ArrayList<ISubscriberRuntime>();
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
@@ -49,8 +51,4 @@
public String toString() {
return "SubscribableRuntime" + " [" + feedId + "]" + "(" + runtimeId + ")";
}
-
- public FeedRuntimeType getFeedRuntimeType() {
- return runtimeId.getFeedRuntimeType();
- }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java
index b69a7b3..82cdddf 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java
@@ -21,13 +21,18 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.active.ActiveJob;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.api.IFeedJoint;
import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.util.FeedUtils.JobType;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
-public class FeedConnectJobInfo extends FeedJobInfo {
+public class FeedConnectJobInfo extends ActiveJob {
+ private static final long serialVersionUID = 1L;
private final FeedConnectionId connectionId;
private final Map<String, String> feedPolicy;
private final IFeedJoint sourceFeedJoint;
@@ -38,10 +43,10 @@
private List<String> storageLocations;
private int partitionStarts = 0;
- public FeedConnectJobInfo(JobId jobId, FeedJobState state, FeedConnectionId connectionId,
+ public FeedConnectJobInfo(EntityId entityId, JobId jobId, ActivityState state, FeedConnectionId connectionId,
IFeedJoint sourceFeedJoint, IFeedJoint computeFeedJoint, JobSpecification spec,
Map<String, String> feedPolicy) {
- super(jobId, state, FeedJobInfo.JobType.FEED_CONNECT, spec);
+ super(entityId, jobId, state, JobType.FEED_CONNECT, spec);
this.connectionId = connectionId;
this.sourceFeedJoint = sourceFeedJoint;
this.computeFeedJoint = computeFeedJoint;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java
index 3b11811..4114e82 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java
@@ -20,27 +20,31 @@
import java.util.List;
+import org.apache.asterix.active.ActiveJob;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.util.FeedUtils.JobType;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobSpecification;
-public class FeedIntakeInfo extends FeedJobInfo {
+public class FeedIntakeInfo extends ActiveJob {
- private final FeedId feedId;
+ private static final long serialVersionUID = 1L;
+ private final EntityId feedId;
private final IFeedJoint intakeFeedJoint;
private final JobSpecification spec;
private List<String> intakeLocation;
- public FeedIntakeInfo(JobId jobId, FeedJobState state, JobType jobType, FeedId feedId, IFeedJoint intakeFeedJoint,
+ public FeedIntakeInfo(JobId jobId, ActivityState state, EntityId feedId, IFeedJoint intakeFeedJoint,
JobSpecification spec) {
- super(jobId, state, FeedJobInfo.JobType.INTAKE, spec);
+ super(feedId, jobId, state, JobType.INTAKE, spec);
this.feedId = feedId;
this.intakeFeedJoint = intakeFeedJoint;
this.spec = spec;
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
@@ -48,6 +52,7 @@
return intakeFeedJoint;
}
+ @Override
public JobSpecification getSpec() {
return spec;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedJobInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedJobInfo.java
deleted file mode 100644
index 92e00cb..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedJobInfo.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.watch;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedJobInfo {
-
- private static final Logger LOGGER = Logger.getLogger(FeedJobInfo.class.getName());
-
- public enum JobType {
- INTAKE,
- FEED_CONNECT
- }
-
- public enum FeedJobState {
- CREATED,
- ACTIVE,
- UNDER_RECOVERY,
- ENDED
- }
-
- protected final JobId jobId;
- protected final JobType jobType;
- protected FeedJobState state;
- protected JobSpecification spec;
-
- public FeedJobInfo(JobId jobId, FeedJobState state, JobType jobType, JobSpecification spec) {
- this.jobId = jobId;
- this.state = state;
- this.jobType = jobType;
- this.spec = spec;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public FeedJobState getState() {
- return state;
- }
-
- public void setState(FeedJobState state) {
- this.state = state;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(this + " is in " + state + " state.");
- }
- }
-
- public JobType getJobType() {
- return jobType;
- }
-
- public JobSpecification getSpec() {
- return spec;
- }
-
- public void setSpec(JobSpecification spec) {
- this.spec = spec;
- }
-
- public String toString() {
- return jobId + " [" + jobType + "]";
- }
-
-}
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 36098ee..84c2cb4 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
@@ -20,13 +20,13 @@
import java.util.Map;
+import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedManager;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -56,13 +56,13 @@
private final Map<String, String> feedPolicyProperties;
/** The source feed from which the feed derives its data from. **/
- private final FeedId sourceFeedId;
+ private final EntityId sourceFeedId;
/** The subscription location at which the recipient feed receives tuples from the source feed {SOURCE_FEED_INTAKE_STAGE , SOURCE_FEED_COMPUTE_STAGE} **/
private final FeedRuntimeType subscriptionLocation;
- public FeedCollectOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId, FeedId sourceFeedId,
- ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicyProperties,
+ public FeedCollectOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId,
+ EntityId sourceFeedId, ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicyProperties,
FeedRuntimeType subscriptionLocation) {
super(spec, 0, 1);
this.recordDescriptors[0] = rDesc;
@@ -77,13 +77,11 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
- FeedManager feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
+ ActiveManager feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
.getApplicationContext().getApplicationObject()).getFeedManager();
- FeedRuntimeId sourceRuntimeId =
- new FeedRuntimeId(sourceFeedId, subscriptionLocation, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
- ISubscribableRuntime sourceRuntime = feedManager.getSubscribableRuntime(sourceRuntimeId);
- return new FeedCollectOperatorNodePushable(ctx, sourceFeedId, connectionId, feedPolicyProperties, partition,
- nPartitions, sourceRuntime);
+ ActiveRuntimeId sourceRuntimeId = new ActiveRuntimeId(sourceFeedId, subscriptionLocation.toString(), partition);
+ ISubscribableRuntime sourceRuntime = (ISubscribableRuntime) feedManager.getSubscribableRuntime(sourceRuntimeId);
+ return new FeedCollectOperatorNodePushable(ctx, connectionId, feedPolicyProperties, partition, sourceRuntime);
}
public FeedConnectionId getFeedConnectionId() {
@@ -102,7 +100,7 @@
return recordDescriptors[0];
}
- public FeedId getSourceFeedId() {
+ public EntityId getSourceFeedId() {
return sourceFeedId;
}
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 aeea6ba..231fe99 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
@@ -20,19 +20,18 @@
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.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedManager;
-import org.apache.asterix.external.feed.message.FeedPartitionStartMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.runtime.CollectionRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -47,46 +46,46 @@
private final FeedConnectionId connectionId;
private final Map<String, String> feedPolicy;
private final FeedPolicyAccessor policyAccessor;
- private final FeedManager feedManager;
+ private final ActiveManager feedManager;
private final ISubscribableRuntime sourceRuntime;
private final IHyracksTaskContext ctx;
private CollectionRuntime collectRuntime;
- public FeedCollectOperatorNodePushable(IHyracksTaskContext ctx, FeedId sourceFeedId,
- FeedConnectionId feedConnectionId, Map<String, String> feedPolicy, int partition, int nPartitions,
- ISubscribableRuntime sourceRuntime) {
+ public FeedCollectOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedConnectionId,
+ Map<String, String> feedPolicy, int partition, ISubscribableRuntime sourceRuntime) {
this.ctx = ctx;
this.partition = partition;
this.connectionId = feedConnectionId;
this.sourceRuntime = sourceRuntime;
this.feedPolicy = feedPolicy;
this.policyAccessor = new FeedPolicyAccessor(feedPolicy);
- this.feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ this.feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
.getApplicationObject()).getFeedManager();
}
@Override
public void initialize() throws HyracksDataException {
try {
- FeedRuntimeId runtimeId = new FeedRuntimeId(connectionId.getFeedId(), FeedRuntimeType.COLLECT, partition,
- FeedRuntimeId.DEFAULT_TARGET_ID);
+ ActiveRuntimeId runtimeId =
+ new ActiveRuntimeId(connectionId.getFeedId(), FeedRuntimeType.COLLECT.toString(), partition);
// Does this collector have a handler?
FrameTupleAccessor tAccessor = new FrameTupleAccessor(recordDesc);
if (policyAccessor.bufferingEnabled()) {
writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, writer, policyAccessor, tAccessor,
- feedManager.getFeedMemoryManager());
+ feedManager.getFramePool());
} else {
writer = new SyncFeedRuntimeInputHandler(ctx, writer, tAccessor);
}
collectRuntime = new CollectionRuntime(connectionId, runtimeId, sourceRuntime, feedPolicy, ctx,
new FeedFrameCollector(policyAccessor, writer, connectionId));
- feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, collectRuntime);
+ feedManager.getActiveRuntimeRegistry().registerRuntime(collectRuntime);
sourceRuntime.subscribe(collectRuntime);
// Notify CC that Collection started
ctx.sendApplicationMessageToCC(
- new FeedPartitionStartMessage(connectionId.getFeedId(), ctx.getJobletContext().getJobId()), null);
+ new ActivePartitionMessage(connectionId.getFeedId(), ctx.getJobletContext().getJobId(), null),
+ null);
collectRuntime.waitTillCollectionOver();
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, collectRuntime.getRuntimeId());
+ feedManager.getActiveRuntimeRegistry().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/FeedIntakeOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
index 69aa59a..f4ea60f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
@@ -21,11 +21,12 @@
import java.util.Map;
import java.util.logging.Logger;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.om.types.ARecordType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -47,7 +48,7 @@
private static final Logger LOGGER = Logger.getLogger(FeedIntakeOperatorDescriptor.class.getName());
/** The unique identifier of the feed that is being ingested. **/
- private final FeedId feedId;
+ private final EntityId feedId;
private final FeedPolicyAccessor policyAccessor;
@@ -71,7 +72,8 @@
public FeedIntakeOperatorDescriptor(JobSpecification spec, IFeed primaryFeed, IAdapterFactory adapterFactory,
ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor, RecordDescriptor rDesc) {
super(spec, 0, 1);
- this.feedId = new FeedId(primaryFeed.getDataverseName(), primaryFeed.getFeedName());
+ this.feedId = new EntityId(FeedConnectionId.FEED_EXTENSION_NAME, primaryFeed.getDataverseName(),
+ primaryFeed.getFeedName());
this.adaptorFactory = adapterFactory;
this.adapterOutputType = adapterOutputType;
this.policyAccessor = policyAccessor;
@@ -82,7 +84,8 @@
String adapterFactoryClassName, ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor,
RecordDescriptor rDesc) {
super(spec, 0, 1);
- this.feedId = new FeedId(primaryFeed.getDataverseName(), primaryFeed.getFeedName());
+ this.feedId = new EntityId(FeedConnectionId.FEED_EXTENSION_NAME, primaryFeed.getDataverseName(),
+ primaryFeed.getFeedName());
this.adaptorFactoryClassName = adapterFactoryClassName;
this.adaptorLibraryName = adapterLibraryName;
this.adaptorConfiguration = primaryFeed.getAdapterConfiguration();
@@ -124,7 +127,7 @@
return adapterFactory;
}
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
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 04ef016..ffa451b 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
@@ -18,22 +18,21 @@
*/
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.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedManager;
-import org.apache.asterix.external.feed.message.FeedPartitionStartMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.runtime.AdapterRuntimeManager;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.feed.runtime.IngestionRuntime;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
/**
@@ -43,13 +42,13 @@
*/
public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
- private final FeedId feedId;
+ private final EntityId feedId;
private final int partition;
private final IHyracksTaskContext ctx;
private final IAdapterFactory adapterFactory;
private final FeedIntakeOperatorDescriptor opDesc;
- public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, IAdapterFactory adapterFactory,
+ public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, EntityId feedId, IAdapterFactory adapterFactory,
int partition, FeedPolicyAccessor policyAccessor, IRecordDescriptorProvider recordDescProvider,
FeedIntakeOperatorDescriptor feedIntakeOperatorDescriptor) {
this.opDesc = feedIntakeOperatorDescriptor;
@@ -62,7 +61,7 @@
@Override
public void initialize() throws HyracksDataException {
- FeedManager feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
+ ActiveManager feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
.getApplicationContext().getApplicationObject()).getFeedManager();
AdapterRuntimeManager adapterRuntimeManager = null;
DistributeFeedFrameWriter frameDistributor = null;
@@ -73,17 +72,15 @@
// create the adapter
FeedAdapter adapter = (FeedAdapter) adapterFactory.createAdapter(ctx, partition);
// create the distributor
- frameDistributor = new DistributeFeedFrameWriter(ctx, feedId, writer, FeedRuntimeType.INTAKE, partition,
- new FrameTupleAccessor(recordDesc));
+ frameDistributor = new DistributeFeedFrameWriter(feedId, writer, FeedRuntimeType.INTAKE, partition);
// create adapter runtime manager
adapterRuntimeManager = new AdapterRuntimeManager(feedId, adapter, frameDistributor, partition);
// create and register the runtime
- FeedRuntimeId runtimeId =
- new FeedRuntimeId(feedId, FeedRuntimeType.INTAKE, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ActiveRuntimeId runtimeId = new ActiveRuntimeId(feedId, FeedRuntimeType.INTAKE.toString(), partition);
ingestionRuntime = new IngestionRuntime(feedId, runtimeId, frameDistributor, adapterRuntimeManager, ctx);
- feedManager.registerFeedSubscribableRuntime(ingestionRuntime);
+ feedManager.registerRuntime(ingestionRuntime);
// Notify FeedJobNotificationHandler that this provider is ready to receive subscription requests.
- ctx.sendApplicationMessageToCC(new FeedPartitionStartMessage(feedId, ctx.getJobletContext().getJobId()),
+ ctx.sendApplicationMessageToCC(new ActivePartitionMessage(feedId, ctx.getJobletContext().getJobId(), null),
null);
// open the distributor
open = true;
@@ -95,7 +92,7 @@
}
}
// The ingestion is over. we need to remove the runtime from the manager
- feedManager.deregisterFeedSubscribableRuntime(ingestionRuntime.getRuntimeId());
+ feedManager.deregisterRuntime(ingestionRuntime.getRuntimeId());
// If there was a failure, we need to throw an exception
if (adapterRuntimeManager.isFailed()) {
throw new HyracksDataException("Unable to ingest data");
@@ -108,7 +105,7 @@
*/
if (ingestionRuntime != null) {
ingestionRuntime.terminate();
- feedManager.deregisterFeedSubscribableRuntime(ingestionRuntime.getRuntimeId());
+ feedManager.deregisterRuntime(ingestionRuntime.getRuntimeId());
}
throw new HyracksDataException(ie);
} finally {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java
index 219110f..61451b1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.external.operators;
-import org.apache.asterix.external.feed.api.IFeedMessage;
+import org.apache.asterix.active.IActiveMessage;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -28,17 +28,20 @@
import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
/**
- * Sends a control message to the registered message queue for feed specified by its feedId.
+ * @deprecated
+ * Sends a control message to the registered message queue for feed specified by its feedId.
+ * For messaging, use IMessageBroker interfaces
*/
+@Deprecated
public class FeedMessageOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
private static final long serialVersionUID = 1L;
private final FeedConnectionId connectionId;
- private final IFeedMessage feedMessage;
+ private final IActiveMessage feedMessage;
public FeedMessageOperatorDescriptor(JobSpecification spec, FeedConnectionId connectionId,
- IFeedMessage feedMessage) {
+ IActiveMessage feedMessage) {
super(spec, 0, 1);
this.connectionId = connectionId;
this.feedMessage = feedMessage;
@@ -47,7 +50,7 @@
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
- return new FeedMessageOperatorNodePushable(ctx, connectionId, feedMessage, partition, nPartitions);
+ return new FeedMessageOperatorNodePushable(ctx, connectionId, feedMessage, partition);
}
}
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 82bf1da..5f92327 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
@@ -21,46 +21,49 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveMessage;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedMessage;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.message.EndFeedMessage;
import org.apache.asterix.external.feed.runtime.AdapterRuntimeManager;
import org.apache.asterix.external.feed.runtime.CollectionRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.feed.runtime.IngestionRuntime;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
/**
- * Runtime for the FeedMessageOpertorDescriptor. This operator is responsible for communicating
- * a feed message to the local feed manager on the host node controller.
+ * @deprecated
+ * Runtime for the FeedMessageOpertorDescriptor. This operator is responsible for communicating
+ * a feed message to the local feed manager on the host node controller.
+ * For messages, use IMessageBroker interfaces
* @see FeedMessageOperatorDescriptor
* IFeedMessage
* IFeedManager
*/
+@Deprecated
public class FeedMessageOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
private static final Logger LOGGER = Logger.getLogger(FeedMessageOperatorNodePushable.class.getName());
private final FeedConnectionId connectionId;
- private final IFeedMessage message;
- private final FeedManager feedManager;
+ private final IActiveMessage message;
+ private final ActiveManager feedManager;
private final int partition;
public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId connectionId,
- IFeedMessage feedMessage, int partition, int nPartitions) {
+ IActiveMessage feedMessage, int partition) {
this.connectionId = connectionId;
this.message = feedMessage;
this.partition = partition;
IAsterixAppRuntimeContext runtimeCtx =
(IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
- this.feedManager = (FeedManager) runtimeCtx.getFeedManager();
+ this.feedManager = (ActiveManager) runtimeCtx.getFeedManager();
}
@Override
@@ -77,6 +80,8 @@
case DISCONTINUE_SOURCE:
handleDiscontinueFeedTypeMessage(endFeedMessage);
break;
+ default:
+ break;
}
break;
default:
@@ -90,10 +95,11 @@
}
private void handleDiscontinueFeedTypeMessage(EndFeedMessage endFeedMessage) throws Exception {
- FeedId sourceFeedId = endFeedMessage.getSourceFeedId();
- FeedRuntimeId subscribableRuntimeId =
- new FeedRuntimeId(sourceFeedId, FeedRuntimeType.INTAKE, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
- ISubscribableRuntime feedRuntime = feedManager.getSubscribableRuntime(subscribableRuntimeId);
+ EntityId sourceFeedId = endFeedMessage.getSourceFeedId();
+ ActiveRuntimeId subscribableRuntimeId =
+ new ActiveRuntimeId(sourceFeedId, FeedRuntimeType.INTAKE.toString(), partition);
+ ISubscribableRuntime feedRuntime =
+ (ISubscribableRuntime) feedManager.getSubscribableRuntime(subscribableRuntimeId);
AdapterRuntimeManager adapterRuntimeManager = ((IngestionRuntime) feedRuntime).getAdapterRuntimeManager();
adapterRuntimeManager.stop();
if (LOGGER.isLoggable(Level.INFO)) {
@@ -105,12 +111,12 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Ending feed:" + endFeedMessage.getFeedConnectionId());
}
- FeedRuntimeId runtimeId = null;
+ ActiveRuntimeId runtimeId;
FeedRuntimeType subscribableRuntimeType = ((EndFeedMessage) message).getSourceRuntimeType();
if (endFeedMessage.isCompleteDisconnection()) {
// subscribableRuntimeType represents the location at which the feed connection receives
// data
- FeedRuntimeType runtimeType = null;
+ FeedRuntimeType runtimeType;
switch (subscribableRuntimeType) {
case INTAKE:
runtimeType = FeedRuntimeType.COLLECT;
@@ -122,10 +128,9 @@
throw new IllegalStateException("Invalid subscribable runtime type " + subscribableRuntimeType);
}
- runtimeId = new FeedRuntimeId(endFeedMessage.getSourceFeedId(), runtimeType, partition,
- FeedRuntimeId.DEFAULT_TARGET_ID);
+ runtimeId = new ActiveRuntimeId(endFeedMessage.getSourceFeedId(), runtimeType.toString(), partition);
CollectionRuntime feedRuntime =
- (CollectionRuntime) feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
+ (CollectionRuntime) feedManager.getActiveRuntimeRegistry().getRuntime(runtimeId);
if (feedRuntime != null) {
feedRuntime.getSourceRuntime().unsubscribe(feedRuntime);
}
@@ -142,11 +147,14 @@
throw new IllegalStateException("Illegal State, invalid runtime type " + subscribableRuntimeType);
case COMPUTE:
// feed could be primary or secondary, doesn't matter
- FeedRuntimeId feedSubscribableRuntimeId = new FeedRuntimeId(connectionId.getFeedId(),
- FeedRuntimeType.COMPUTE, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
- ISubscribableRuntime feedRuntime = feedManager.getSubscribableRuntime(feedSubscribableRuntimeId);
- CollectionRuntime feedCollectionRuntime = (CollectionRuntime) feedManager.getFeedConnectionManager()
- .getFeedRuntime(connectionId, runtimeId);
+ ActiveRuntimeId feedSubscribableRuntimeId = new ActiveRuntimeId(connectionId.getFeedId(),
+ FeedRuntimeType.COMPUTE.toString(), partition);
+ ISubscribableRuntime feedRuntime =
+ (ISubscribableRuntime) feedManager.getSubscribableRuntime(feedSubscribableRuntimeId);
+ runtimeId = new ActiveRuntimeId(endFeedMessage.getSourceFeedId(),
+ FeedRuntimeType.COMPUTE_COLLECT.toString(), partition);
+ CollectionRuntime feedCollectionRuntime =
+ (CollectionRuntime) feedManager.getActiveRuntimeRegistry().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 8d8bc28..54e17ef 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
@@ -23,17 +23,17 @@
import java.util.logging.Level;
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.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.policy.FeedPolicyEnforcer;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.util.FeedUtils;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IActivity;
@@ -63,7 +63,7 @@
* The Feed Runtime instance associated with the operator. Feed Runtime
* captures the state of the operator while the feed is active.
*/
- private FeedRuntime feedRuntime;
+ private ActiveRuntime feedRuntime;
/**
* A unique identifier for the feed instance. A feed instance represents
@@ -78,7 +78,7 @@
private int partition;
/** The (singleton) instance of IFeedManager **/
- private FeedManager feedManager;
+ private ActiveManager feedManager;
private FrameTupleAccessor fta;
@@ -109,7 +109,7 @@
this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
this.partition = partition;
this.connectionId = feedConnectionId;
- this.feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ this.feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
.getApplicationObject()).getFeedManager();
this.message = new VSizeFrame(ctx);
ctx.setSharedObject(message);
@@ -119,8 +119,7 @@
@Override
public void open() throws HyracksDataException {
- FeedRuntimeId runtimeId =
- new FeedRuntimeId(connectionId.getFeedId(), runtimeType, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ActiveRuntimeId runtimeId = new ActiveRuntimeId(connectionId.getFeedId(), runtimeType.toString(), partition);
try {
initializeNewFeedRuntime(runtimeId);
opened = true;
@@ -131,18 +130,18 @@
}
}
- private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ private void initializeNewFeedRuntime(ActiveRuntimeId runtimeId) throws Exception {
fta = new FrameTupleAccessor(recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0));
FeedPolicyAccessor fpa = policyEnforcer.getFeedPolicyAccessor();
coreOperator.setOutputFrameWriter(0, writer, recordDesc);
if (fpa.bufferingEnabled()) {
writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator, fpa, fta,
- feedManager.getFeedMemoryManager());
+ feedManager.getFramePool());
} else {
writer = new SyncFeedRuntimeInputHandler(ctx, coreOperator, fta);
}
- feedRuntime = new FeedRuntime(runtimeId);
- feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, feedRuntime);
+ feedRuntime = new ActiveRuntime(runtimeId);
+ feedManager.getActiveRuntimeRegistry().registerRuntime(feedRuntime);
}
@Override
@@ -173,7 +172,7 @@
}
private void deregister() {
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, feedRuntime.getRuntimeId());
+ feedManager.getActiveRuntimeRegistry().deregisterRuntime(feedRuntime.getRuntimeId());
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java
index c7dd3d2..908601d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java
@@ -20,8 +20,8 @@
import java.util.Map;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
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 47df39e..6f679f7 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
@@ -23,17 +23,17 @@
import java.util.logging.Level;
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;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.policy.FeedPolicyEnforcer;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
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;
@@ -61,7 +61,7 @@
* The Feed Runtime instance associated with the operator. Feed Runtime
* captures the state of the operator while the feed is active.
*/
- private FeedRuntime feedRuntime;
+ private ActiveRuntime feedRuntime;
/**
* A unique identifier for the feed instance. A feed instance represents
@@ -79,7 +79,7 @@
private final FeedRuntimeType runtimeType = FeedRuntimeType.STORE;
/** The (singleton) instance of IFeedManager **/
- private final FeedManager feedManager;
+ private final ActiveManager feedManager;
private FrameTupleAccessor fta;
@@ -103,7 +103,7 @@
this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
this.partition = partition;
this.connectionId = feedConnectionId;
- this.feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ this.feedManager = (ActiveManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
.getApplicationObject()).getFeedManager();
this.targetId = targetId;
this.message = new VSizeFrame(ctx);
@@ -114,7 +114,8 @@
@Override
public void open() throws HyracksDataException {
- FeedRuntimeId runtimeId = new FeedRuntimeId(connectionId.getFeedId(), runtimeType, partition, targetId);
+ ActiveRuntimeId runtimeId =
+ new ActiveRuntimeId(connectionId.getFeedId(), runtimeType.toString() + "." + targetId, partition);
try {
initializeNewFeedRuntime(runtimeId);
insertOperator.open();
@@ -124,7 +125,7 @@
}
}
- private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ private void initializeNewFeedRuntime(ActiveRuntimeId runtimeId) throws Exception {
fta = new FrameTupleAccessor(recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0));
insertOperator.setOutputFrameWriter(0, writer, recordDesc);
if (insertOperator instanceof AsterixLSMInsertDeleteOperatorNodePushable) {
@@ -138,7 +139,7 @@
}
if (policyEnforcer.getFeedPolicyAccessor().bufferingEnabled()) {
writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, insertOperator,
- policyEnforcer.getFeedPolicyAccessor(), fta, feedManager.getFeedMemoryManager());
+ policyEnforcer.getFeedPolicyAccessor(), fta, feedManager.getFramePool());
} else {
writer = new SyncFeedRuntimeInputHandler(ctx, insertOperator, fta);
}
@@ -146,9 +147,10 @@
}
private void setupBasicRuntime(IFrameWriter frameWriter) throws Exception {
- FeedRuntimeId runtimeId = new FeedRuntimeId(connectionId.getFeedId(), runtimeType, partition, targetId);
- feedRuntime = new FeedRuntime(runtimeId);
- feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, feedRuntime);
+ ActiveRuntimeId runtimeId =
+ new ActiveRuntimeId(connectionId.getFeedId(), runtimeType.toString() + "." + targetId, partition);
+ feedRuntime = new ActiveRuntime(runtimeId);
+ feedManager.getActiveRuntimeRegistry().registerRuntime(feedRuntime);
}
@Override
@@ -177,7 +179,7 @@
}
private void deregister() {
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, feedRuntime.getRuntimeId());
+ feedManager.getActiveRuntimeRegistry().deregisterRuntime(feedRuntime.getRuntimeId());
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
index 8228c39..6b7eb31 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
@@ -42,6 +42,29 @@
import org.apache.hyracks.util.IntSerDeUtils;
public class FeedUtils {
+
+ public enum JobType {
+ INTAKE,
+ FEED_CONNECT
+ }
+
+ public enum FeedRuntimeType {
+ INTAKE,
+ COLLECT,
+ COMPUTE_COLLECT,
+ COMPUTE,
+ STORE,
+ OTHER,
+ ETS,
+ JOIN
+ }
+
+ public enum Mode {
+ PROCESS, // There is memory
+ SPILL, // Memory budget has been consumed. Now we're writing to disk
+ DISCARD // Memory and Disk space budgets have been consumed. Now we're discarding
+ }
+
private static String prepareDataverseFeedName(String dataverseName, String feedName) {
return dataverseName + File.separator + feedName;
}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/ConcurrentFramePoolUnitTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/ConcurrentFramePoolUnitTest.java
index 49042b8..0f6a2ea 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/ConcurrentFramePoolUnitTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/ConcurrentFramePoolUnitTest.java
@@ -24,9 +24,9 @@
import java.util.Random;
import java.util.concurrent.LinkedBlockingDeque;
+import org.apache.asterix.active.ConcurrentFramePool;
+import org.apache.asterix.active.FrameAction;
import org.apache.asterix.common.config.AsterixFeedProperties;
-import org.apache.asterix.external.feed.dataflow.FrameAction;
-import org.apache.asterix.external.feed.management.ConcurrentFramePool;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.junit.Assert;
import org.mockito.Mockito;
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
index bc1c328..e643206 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
@@ -25,13 +25,13 @@
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.ConcurrentFramePool;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.management.ConcurrentFramePool;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
+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;
@@ -72,10 +72,9 @@
private FeedRuntimeInputHandler createInputHandler(IHyracksTaskContext ctx, IFrameWriter writer,
FeedPolicyAccessor fpa, ConcurrentFramePool framePool) throws HyracksDataException {
FrameTupleAccessor fta = Mockito.mock(FrameTupleAccessor.class);
- FeedId feedId = new FeedId(DATAVERSE, FEED);
+ EntityId feedId = new EntityId(FeedConnectionId.FEED_EXTENSION_NAME, DATAVERSE, FEED);
FeedConnectionId connectionId = new FeedConnectionId(feedId, DATASET);
- FeedRuntimeId runtimeId =
- new FeedRuntimeId(feedId, FeedRuntimeType.COLLECT, 0, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ActiveRuntimeId runtimeId = new ActiveRuntimeId(feedId, FeedRuntimeType.COLLECT.toString(), 0);
return new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, writer, fpa, fta, framePool);
}
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 092bf69..14f229a 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
@@ -24,11 +24,11 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.watch.FeedActivity;
import org.apache.asterix.external.util.ExternalDataConstants;
@@ -52,7 +52,7 @@
* Represents the AQL statement for subscribing to a feed.
* This AQL statement is private and may not be used by the end-user.
*/
-public class SubscribeFeedStatement implements Statement {
+public class SubscribeFeedStatement extends Statement {
private static final Logger LOGGER = Logger.getLogger(SubscribeFeedStatement.class.getName());
private final FeedConnectionRequest connectionRequest;
@@ -71,10 +71,10 @@
public void initialize(MetadataTransactionContext mdTxnCtx) throws MetadataException {
this.query = new Query();
- FeedId sourceFeedId = connectionRequest.getFeedJointKey().getFeedId();
- Feed subscriberFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx,
- connectionRequest.getReceivingFeedId().getDataverse(),
- connectionRequest.getReceivingFeedId().getFeedName());
+ EntityId sourceFeedId = connectionRequest.getFeedJointKey().getFeedId();
+ Feed subscriberFeed =
+ MetadataManager.INSTANCE.getFeed(mdTxnCtx, connectionRequest.getReceivingFeedId().getDataverse(),
+ connectionRequest.getReceivingFeedId().getEntityName());
if (subscriberFeed == null) {
throw new IllegalStateException(" Subscriber feed " + subscriberFeed + " not found.");
}
@@ -100,8 +100,9 @@
builder.append("insert into dataset " + connectionRequest.getTargetDataset() + " ");
builder.append(" (" + " for $x in feed-collect ('" + sourceFeedId.getDataverse() + "'" + "," + "'"
- + sourceFeedId.getFeedName() + "'" + "," + "'" + connectionRequest.getReceivingFeedId().getFeedName()
- + "'" + "," + "'" + connectionRequest.getSubscriptionLocation().name() + "'" + "," + "'"
+ + sourceFeedId.getEntityName() + "'" + "," + "'"
+ + connectionRequest.getReceivingFeedId().getEntityName() + "'" + "," + "'"
+ + connectionRequest.getSubscriptionLocation().name() + "'" + "," + "'"
+ connectionRequest.getTargetDataset() + "'" + "," + "'" + feedOutputType + "'" + ")");
List<String> functionsToApply = connectionRequest.getFunctionsToApply();
@@ -156,8 +157,8 @@
}
@Override
- public Kind getKind() {
- return Kind.SUBSCRIBE_FEED;
+ public byte getKind() {
+ return Statement.SUBSCRIBE_FEED;
}
public String getPolicy() {
@@ -179,8 +180,8 @@
private String getOutputType(MetadataTransactionContext mdTxnCtx) throws MetadataException {
String outputType = null;
- FeedId feedId = connectionRequest.getReceivingFeedId();
- Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getFeedName());
+ EntityId feedId = connectionRequest.getReceivingFeedId();
+ Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName());
FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(connectionRequest.getPolicyParameters());
try {
switch (feed.getFeedType()) {
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
index 2fc4b14..f30d4a6 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
@@ -70,7 +70,7 @@
}
// Translate the query into a Range Map
- if (hintStatements.get(0).getKind() != Statement.Kind.QUERY) {
+ if (hintStatements.get(0).getKind() != Statement.QUERY) {
throw new AsterixException("Not a proper query for the range hint.");
}
Query q = (Query) hintStatements.get(0);
@@ -151,8 +151,8 @@
int fieldIndex = 0;
int fieldType = rangeMap.getTag(0, 0);
AqlBinaryComparatorFactoryProvider comparatorFactory = AqlBinaryComparatorFactoryProvider.INSTANCE;
- IBinaryComparatorFactory bcf = comparatorFactory
- .getBinaryComparatorFactory(ATypeTag.VALUE_TYPE_MAPPING[fieldType], ascending);
+ IBinaryComparatorFactory bcf =
+ comparatorFactory.getBinaryComparatorFactory(ATypeTag.VALUE_TYPE_MAPPING[fieldType], ascending);
IBinaryComparator comparator = bcf.createBinaryComparator();
int c = 0;
for (int split = 1; split < rangeMap.getSplitCount(); ++split) {
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 3184d1e..6f0f1f1 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
@@ -18,45 +18,52 @@
*/
package org.apache.asterix.lang.common.base;
-public interface Statement extends ILangExpression {
- public enum Kind {
- DATASET_DECL,
- DATAVERSE_DECL,
- DATAVERSE_DROP,
- DATASET_DROP,
- DELETE,
- INSERT,
- UPSERT,
- UPDATE,
- DML_CMD_LIST,
- FUNCTION_DECL,
- LOAD,
- NODEGROUP_DECL,
- NODEGROUP_DROP,
- QUERY,
- SET,
- TYPE_DECL,
- TYPE_DROP,
- WRITE,
- CREATE_INDEX,
- INDEX_DECL,
- CREATE_DATAVERSE,
- INDEX_DROP,
- CREATE_PRIMARY_FEED,
- CREATE_SECONDARY_FEED,
- DROP_FEED,
- CONNECT_FEED,
- DISCONNECT_FEED,
- SUBSCRIBE_FEED,
- CREATE_FEED_POLICY,
- DROP_FEED_POLICY,
- CREATE_FUNCTION,
- FUNCTION_DROP,
- COMPACT,
- EXTERNAL_DATASET_REFRESH,
- RUN
- }
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
- public abstract Kind getKind();
+public abstract class Statement implements ILangExpression {
+ public static final byte DATASET_DECL = 0x00;
+ public static final byte DATAVERSE_DECL = 0x01;
+ public static final byte DATAVERSE_DROP = 0x02;
+ public static final byte DATASET_DROP = 0x03;
+ public static final byte DELETE = 0x04;
+ public static final byte INSERT = 0x05;
+ public static final byte UPSERT = 0x06;
+ public static final byte UPDATE = 0x07;
+ public static final byte DML_CMD_LIST = 0x08;
+ public static final byte FUNCTION_DECL = 0x09;
+ public static final byte LOAD = 0x0a;
+ public static final byte NODEGROUP_DECL = 0x0b;
+ public static final byte NODEGROUP_DROP = 0x0c;
+ public static final byte QUERY = 0x0d;
+ public static final byte SET = 0x0e;
+ public static final byte TYPE_DECL = 0x0f;
+ public static final byte TYPE_DROP = 0x10;
+ public static final byte WRITE = 0x11;
+ public static final byte CREATE_INDEX = 0x12;
+ public static final byte INDEX_DECL = 0x13;
+ public static final byte CREATE_DATAVERSE = 0x14;
+ public static final byte INDEX_DROP = 0x15;
+ public static final byte CREATE_PRIMARY_FEED = 0x16;
+ public static final byte CREATE_SECONDARY_FEED = 0x17;
+ public static final byte DROP_FEED = 0x18;
+ public static final byte CONNECT_FEED = 0x19;
+ public static final byte DISCONNECT_FEED = 0x1a;
+ public static final byte SUBSCRIBE_FEED = 0x1b;
+ public static final byte CREATE_FEED_POLICY = 0x1c;
+ public static final byte DROP_FEED_POLICY = 0x1d;
+ public static final byte CREATE_FUNCTION = 0x1e;
+ public static final byte FUNCTION_DROP = 0x1f;
+ public static final byte COMPACT = 0x20;
+ public static final byte EXTERNAL_DATASET_REFRESH = 0x21;
+ public static final byte RUN = 0x22;
+ public static final List<Byte> VALUES = Collections.unmodifiableList(
+ Arrays.asList(DATASET_DECL, DATAVERSE_DECL, DATAVERSE_DROP, DATASET_DROP, DELETE, INSERT, UPSERT, UPDATE,
+ DML_CMD_LIST, FUNCTION_DECL, LOAD, NODEGROUP_DECL, NODEGROUP_DROP, QUERY, SET, TYPE_DECL, TYPE_DROP,
+ WRITE, CREATE_INDEX, INDEX_DECL, CREATE_DATAVERSE, INDEX_DROP, CREATE_PRIMARY_FEED,
+ CREATE_SECONDARY_FEED, DROP_FEED, CONNECT_FEED, DISCONNECT_FEED, SUBSCRIBE_FEED, CREATE_FEED_POLICY,
+ DROP_FEED_POLICY, CREATE_FUNCTION, FUNCTION_DROP, COMPACT, EXTERNAL_DATASET_REFRESH, RUN));
+ public abstract byte getKind();
}
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 93d151c..531957f 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class CompactStatement implements Statement {
+public class CompactStatement extends Statement {
private final Identifier dataverseName;
private final Identifier datasetName;
@@ -34,8 +34,8 @@
}
@Override
- public Kind getKind() {
- return Kind.COMPACT;
+ public byte getKind() {
+ return Statement.COMPACT;
}
public Identifier getDataverseName() {
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 b4208b9..33e3340 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
@@ -25,7 +25,7 @@
import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
import org.apache.hyracks.algebricks.common.utils.Pair;
-public class ConnectFeedStatement implements Statement {
+public class ConnectFeedStatement extends Statement {
private final Identifier dataverseName;
private final Identifier datasetName;
@@ -77,8 +77,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CONNECT_FEED;
+ public byte getKind() {
+ return Statement.CONNECT_FEED;
}
public String getPolicy() {
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 1eb8372..820ae5f 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class CreateDataverseStatement implements Statement {
+public class CreateDataverseStatement extends Statement {
private Identifier dataverseName;
private String format;
@@ -31,10 +31,11 @@
public CreateDataverseStatement(Identifier dataverseName, String format, boolean ifNotExists) {
this.dataverseName = dataverseName;
- if (format == null)
+ if (format == null) {
this.format = "org.apache.asterix.runtime.formats.NonTaggedDataFormat";
- else
+ } else {
this.format = format;
+ }
this.ifNotExists = ifNotExists;
}
@@ -51,8 +52,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_DATAVERSE;
+ public byte getKind() {
+ return Statement.CREATE_DATAVERSE;
}
@Override
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 bd3192c..e972cad 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
@@ -24,7 +24,7 @@
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class CreateFeedPolicyStatement implements Statement {
+public class CreateFeedPolicyStatement extends Statement {
private final String policyName;
private final String sourcePolicyName;
@@ -58,8 +58,8 @@
}
@Override
- public Kind getKind() {
- return Statement.Kind.CREATE_FEED_POLICY;
+ public byte getKind() {
+ return Statement.CREATE_FEED_POLICY;
}
@Override
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 53d05d2..9635836 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
@@ -25,7 +25,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
-public abstract class CreateFeedStatement implements Statement {
+public abstract class CreateFeedStatement extends Statement {
private final Pair<Identifier, Identifier> qName;
private final FunctionSignature appliedFunction;
@@ -55,9 +55,6 @@
}
@Override
- public abstract Kind getKind();
-
- @Override
public abstract <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws AsterixException;
}
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 4ab1fe6..f7d0363 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
@@ -27,7 +27,7 @@
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class CreateFunctionStatement implements Statement {
+public class CreateFunctionStatement extends Statement {
private final FunctionSignature signature;
private final String functionBody;
@@ -58,8 +58,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_FUNCTION;
+ public byte getKind() {
+ return Statement.CREATE_FUNCTION;
}
public List<String> getParamList() {
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 44c249f..a2337e1 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
@@ -29,7 +29,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
-public class CreateIndexStatement implements Statement {
+public class CreateIndexStatement extends Statement {
private Identifier indexName;
private Identifier dataverseName;
@@ -119,8 +119,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_INDEX;
+ public byte getKind() {
+ return Statement.CREATE_INDEX;
}
public boolean hasMetaField() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
index dbcbc72..9dac0c4 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
@@ -27,7 +27,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
-public class CreatePrimaryFeedStatement extends CreateFeedStatement implements Statement {
+public class CreatePrimaryFeedStatement extends CreateFeedStatement {
private final String adaptorName;
private final Map<String, String> adaptorConfiguration;
@@ -48,8 +48,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_PRIMARY_FEED;
+ public byte getKind() {
+ return Statement.CREATE_PRIMARY_FEED;
}
@Override
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java
index a4f267f..7ef0a56 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java
@@ -29,7 +29,7 @@
* Represents the AQL statement for creating a secondary feed.
* A secondary feed is one that derives its data from another (primary/secondary) feed.
*/
-public class CreateSecondaryFeedStatement extends CreateFeedStatement implements Statement {
+public class CreateSecondaryFeedStatement extends CreateFeedStatement {
/** The source feed that provides data for this secondary feed. */
private final Pair<Identifier, Identifier> sourceQName;
@@ -50,8 +50,8 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_SECONDARY_FEED;
+ public byte getKind() {
+ return Statement.CREATE_SECONDARY_FEED;
}
@Override
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 10cf8db..fa88a9a 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
@@ -27,7 +27,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class DatasetDecl implements Statement {
+public class DatasetDecl extends Statement {
protected final Identifier name;
protected final Identifier dataverse;
protected final Identifier itemTypeDataverse;
@@ -141,8 +141,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DATASET_DECL;
+ public byte getKind() {
+ return Statement.DATASET_DECL;
}
public IDatasetDetailsDecl getDatasetDetailsDecl() {
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 a804955..ae183e3 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class DataverseDecl implements Statement {
+public class DataverseDecl extends Statement {
private Identifier dataverseName;
@@ -36,8 +36,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DATAVERSE_DECL;
+ public byte getKind() {
+ return Statement.DATAVERSE_DECL;
}
@Override
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 538a05a..fe828fa 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class DataverseDropStatement implements Statement {
+public class DataverseDropStatement extends Statement {
private Identifier dataverseName;
private boolean ifExists;
@@ -34,8 +34,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DATAVERSE_DROP;
+ public byte getKind() {
+ return Statement.DATAVERSE_DROP;
}
public Identifier getDataverseName() {
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 ba3440f..c660d68 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
@@ -28,7 +28,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.commons.lang3.ObjectUtils;
-public class DeleteStatement implements Statement {
+public class DeleteStatement extends Statement {
private VariableExpr vars;
private Identifier dataverseName;
@@ -51,8 +51,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DELETE;
+ public byte getKind() {
+ return Statement.DELETE;
}
public VariableExpr getVariableExpr() {
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 860198e..61ccc7a 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
@@ -24,7 +24,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
-public class DisconnectFeedStatement implements Statement {
+public class DisconnectFeedStatement extends Statement {
private final Identifier dataverseName;
private final Identifier feedName;
@@ -61,8 +61,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DISCONNECT_FEED;
+ public byte getKind() {
+ return Statement.DISCONNECT_FEED;
}
@Override
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/DropStatement.java
index a75e3f6..f8becae 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/DropStatement.java
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class DropStatement implements Statement {
+public class DropStatement extends Statement {
private final Identifier dataverseName;
private final Identifier datasetName;
@@ -36,8 +36,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DATASET_DROP;
+ public byte getKind() {
+ return Statement.DATASET_DROP;
}
public Identifier getDataverseName() {
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 ddf44de..88f96ec 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class FeedDropStatement implements Statement {
+public class FeedDropStatement extends Statement {
private final Identifier dataverseName;
private final Identifier feedName;
@@ -36,8 +36,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DROP_FEED;
+ public byte getKind() {
+ return Statement.DROP_FEED;
}
public Identifier getDataverseName() {
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 fdca3aa..73535fa 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class FeedPolicyDropStatement implements Statement {
+public class FeedPolicyDropStatement extends Statement {
private final Identifier dataverseName;
private final Identifier policyName;
@@ -36,8 +36,8 @@
}
@Override
- public Kind getKind() {
- return Kind.DROP_FEED_POLICY;
+ public byte getKind() {
+ return Statement.DROP_FEED_POLICY;
}
public Identifier getDataverseName() {
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 6681075..57a68f5 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
@@ -27,7 +27,7 @@
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class FunctionDecl implements Statement {
+public class FunctionDecl extends Statement {
private FunctionSignature signature;
private List<VarIdentifier> paramList;
private Expression funcBody;
@@ -73,8 +73,8 @@
}
@Override
- public Kind getKind() {
- return Kind.FUNCTION_DECL;
+ public byte getKind() {
+ return Statement.FUNCTION_DECL;
}
@Override
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 5af2f88..9560180 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class FunctionDropStatement implements Statement {
+public class FunctionDropStatement extends Statement {
private final FunctionSignature signature;
private boolean ifExists;
@@ -34,8 +34,8 @@
}
@Override
- public Kind getKind() {
- return Kind.FUNCTION_DROP;
+ public byte getKind() {
+ return Statement.FUNCTION_DROP;
}
public FunctionSignature getFunctionSignature() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDecl.java
index 3ae3155..06def0f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDecl.java
@@ -18,9 +18,11 @@
*/
package org.apache.asterix.lang.common.statement;
+import org.apache.asterix.lang.common.base.Statement;
+
public class IndexDecl extends CreateIndexStatement {
@Override
- public Kind getKind() {
- return Kind.INDEX_DECL;
+ public byte getKind() {
+ return Statement.INDEX_DECL;
}
}
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 f0bb688..e96a9d5 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class IndexDropStatement implements Statement {
+public class IndexDropStatement extends Statement {
private Identifier dataverseName;
private Identifier datasetName;
@@ -39,8 +39,8 @@
}
@Override
- public Kind getKind() {
- return Kind.INDEX_DROP;
+ public byte getKind() {
+ return Statement.INDEX_DROP;
}
public Identifier getDataverseName() {
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 aab0979..6d2c4f7 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
@@ -24,7 +24,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.commons.lang3.ObjectUtils;
-public class InsertStatement implements Statement {
+public class InsertStatement extends Statement {
private final Identifier dataverseName;
private final Identifier datasetName;
@@ -39,8 +39,8 @@
}
@Override
- public Kind getKind() {
- return Kind.INSERT;
+ public byte getKind() {
+ return Statement.INSERT;
}
public Identifier getDataverseName() {
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 b58a7c5..efea76c 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
@@ -25,7 +25,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class LoadStatement implements Statement {
+public class LoadStatement extends Statement {
private Identifier datasetName;
private Identifier dataverseName;
@@ -67,8 +67,8 @@
}
@Override
- public Kind getKind() {
- return Kind.LOAD;
+ public byte getKind() {
+ return Statement.LOAD;
}
public Identifier getDatasetName() {
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 d13d387..5f6a0a2 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class NodeGroupDropStatement implements Statement {
+public class NodeGroupDropStatement extends Statement {
private Identifier nodeGroupName;
private boolean ifExists;
@@ -34,8 +34,8 @@
}
@Override
- public Kind getKind() {
- return Kind.NODEGROUP_DROP;
+ public byte getKind() {
+ return Statement.NODEGROUP_DROP;
}
public Identifier getNodeGroupName() {
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 475d43b..25244ad 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
@@ -25,7 +25,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class NodegroupDecl implements Statement {
+public class NodegroupDecl extends Statement {
private Identifier nodegroupName;
private List<Identifier> nodeControllerNames;
@@ -54,8 +54,8 @@
}
@Override
- public Kind getKind() {
- return Kind.NODEGROUP_DECL;
+ public byte getKind() {
+ return Statement.NODEGROUP_DECL;
}
@Override
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 a9cba21..8a40ae9 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
@@ -27,7 +27,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.commons.lang3.ObjectUtils;
-public class Query implements Statement {
+public class Query extends Statement {
private boolean topLevel = true;
private Expression body;
private int varCounter;
@@ -71,8 +71,8 @@
}
@Override
- public Kind getKind() {
- return Kind.QUERY;
+ public byte getKind() {
+ return Statement.QUERY;
}
@Override
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 19b6605..47c4ca0 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class RefreshExternalDatasetStatement implements Statement {
+public class RefreshExternalDatasetStatement extends Statement {
private Identifier dataverseName;
private Identifier datasetName;
@@ -50,8 +50,8 @@
}
@Override
- public Kind getKind() {
- return Kind.EXTERNAL_DATASET_REFRESH;
+ public byte getKind() {
+ return Statement.EXTERNAL_DATASET_REFRESH;
}
}
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 1d38ff8..bfdc4a0 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
@@ -25,7 +25,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class RunStatement implements Statement {
+public class RunStatement extends Statement {
private String system;
private List<String> parameters;
@@ -74,8 +74,8 @@
}
@Override
- public Kind getKind() {
- return Kind.RUN;
+ public byte getKind() {
+ return Statement.RUN;
}
}
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 5d61cde..044e42a 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
@@ -22,7 +22,7 @@
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class SetStatement implements Statement {
+public class SetStatement extends Statement {
private String propName;
private String propValue;
@@ -41,8 +41,8 @@
}
@Override
- public Kind getKind() {
- return Kind.SET;
+ public byte getKind() {
+ return Statement.SET;
}
@Override
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 e259fd4..6d41640 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
@@ -25,7 +25,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class TypeDecl implements Statement {
+public class TypeDecl extends Statement {
private final Identifier dataverseName;
private final Identifier ident;
@@ -63,8 +63,8 @@
}
@Override
- public Kind getKind() {
- return Kind.TYPE_DECL;
+ public byte getKind() {
+ return Statement.TYPE_DECL;
}
@Override
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 b0c026c..a4bbe96 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class TypeDropStatement implements Statement {
+public class TypeDropStatement extends Statement {
private final Identifier dataverseName;
private Identifier typeName;
@@ -36,8 +36,8 @@
}
@Override
- public Kind getKind() {
- return Kind.TYPE_DROP;
+ public byte getKind() {
+ return Statement.TYPE_DROP;
}
public Identifier getDataverseName() {
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 c61ec0b..da74549 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
@@ -28,7 +28,7 @@
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.commons.lang3.ObjectUtils;
-public class UpdateStatement implements Statement {
+public class UpdateStatement extends Statement {
private VariableExpr vars;
private Expression target;
@@ -43,8 +43,8 @@
}
@Override
- public Kind getKind() {
- return Kind.UPDATE;
+ public byte getKind() {
+ return Statement.UPDATE;
}
public VariableExpr getVariableExpr() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
index fa11930..f562927 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.lang.common.statement;
+import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.struct.Identifier;
public class UpsertStatement extends InsertStatement {
@@ -27,13 +28,13 @@
}
@Override
- public Kind getKind() {
- return Kind.UPSERT;
+ public byte getKind() {
+ return Statement.UPSERT;
}
@Override
public int hashCode() {
- return super.hashCode() + Kind.UPSERT.ordinal();
+ return super.hashCode() + Statement.UPSERT;
}
@Override
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 090b627..1c49b9d 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
@@ -23,7 +23,7 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-public class WriteStatement implements Statement {
+public class WriteStatement extends Statement {
private final Identifier ncName;
private final String fileName;
@@ -48,8 +48,8 @@
}
@Override
- public Kind getKind() {
- return Kind.WRITE;
+ public byte getKind() {
+ return Statement.WRITE;
}
@Override
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 bc70707..48f33ed 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
@@ -20,9 +20,9 @@
import java.util.List;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.util.AsterixClusterProperties;
@@ -34,7 +34,7 @@
public class FeedDataSource extends AqlDataSource {
private final Feed feed;
- private final FeedId sourceFeedId;
+ private final EntityId sourceFeedId;
private final IFeed.FeedType sourceFeedType;
private final FeedRuntimeType location;
private final String targetDataset;
@@ -45,8 +45,9 @@
public FeedDataSource(Feed feed, AqlSourceId id, String targetDataset, IAType itemType, IAType metaType,
List<IAType> pkTypes, List<List<String>> partitioningKeys,
- List<ScalarFunctionCallExpression> keyAccessExpression, FeedId sourceFeedId, IFeed.FeedType sourceFeedType,
- FeedRuntimeType location, String[] locations, INodeDomain domain) throws AlgebricksException {
+ List<ScalarFunctionCallExpression> keyAccessExpression, EntityId sourceFeedId,
+ IFeed.FeedType sourceFeedType, FeedRuntimeType location, String[] locations, INodeDomain domain)
+ throws AlgebricksException {
super(id, itemType, metaType, AqlDataSourceType.FEED, domain);
this.feed = feed;
this.targetDataset = targetDataset;
@@ -73,7 +74,7 @@
return targetDataset;
}
- public FeedId getSourceFeedId() {
+ public EntityId getSourceFeedId() {
return sourceFeedId;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java
index 1f9e589..1343e53 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java
@@ -21,9 +21,9 @@
import java.util.Map;
+import org.apache.asterix.active.EntityId;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.metadata.MetadataCache;
import org.apache.asterix.metadata.api.IMetadataEntity;
@@ -32,9 +32,10 @@
*/
public class Feed implements IMetadataEntity<Feed>, IFeed {
private static final long serialVersionUID = 1L;
+ public static final String EXTENSION_NAME = "Feed";
/** A unique identifier for the feed */
- private FeedId feedId;
+ private EntityId feedId;
/** The function that is to be applied on each incoming feed tuple **/
private FunctionSignature appliedFunction;
/** The type {@code FeedType} associated with the feed. **/
@@ -50,7 +51,7 @@
public Feed(String dataverseName, String feedName, FunctionSignature appliedFunction, IFeed.FeedType feedType,
String sourceFeedName, String adapterName, Map<String, String> configuration) {
- this.feedId = new FeedId(dataverseName, feedName);
+ this.feedId = new EntityId(EXTENSION_NAME, dataverseName, feedName);
this.appliedFunction = appliedFunction;
this.feedType = feedType;
this.displayName = feedType + "(" + feedId + ")";
@@ -60,7 +61,7 @@
}
@Override
- public FeedId getFeedId() {
+ public EntityId getFeedId() {
return feedId;
}
@@ -71,7 +72,7 @@
@Override
public String getFeedName() {
- return feedId.getFeedName();
+ return feedId.getEntityName();
}
@Override
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 6b55d81..12ab634 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
@@ -21,7 +21,6 @@
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -41,15 +40,14 @@
import org.apache.asterix.external.api.IDataSourceAdapter;
import org.apache.asterix.external.api.IDataSourceAdapter.AdapterType;
import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
import org.apache.asterix.external.provider.AdapterFactoryProvider;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.MetadataManager;
@@ -118,7 +116,7 @@
}
public static Feed validateIfFeedExists(String dataverse, String feedName, MetadataTransactionContext ctx)
- throws MetadataException, AsterixException {
+ throws AsterixException {
Feed feed = MetadataManager.INSTANCE.getFeed(ctx, dataverse, feedName);
if (feed == null) {
throw new AsterixException("Unknown source feed: " + feedName);
@@ -149,11 +147,10 @@
Map<OperatorDescriptorId, IOperatorDescriptor> operatorMap = spec.getOperatorMap();
boolean preProcessingRequired = preProcessingRequired(feedConnectionId);
// copy operators
- String operandId = null;
Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID = new HashMap<>();
- FeedMetaOperatorDescriptor metaOp = null;
+ FeedMetaOperatorDescriptor metaOp;
for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorMap.entrySet()) {
- operandId = FeedRuntimeId.DEFAULT_TARGET_ID;
+ String operandId = null;
IOperatorDescriptor opDesc = entry.getValue();
if (opDesc instanceof FeedCollectOperatorDescriptor) {
FeedCollectOperatorDescriptor orig = (FeedCollectOperatorDescriptor) opDesc;
@@ -169,8 +166,8 @@
FeedRuntimeType.STORE, false, operandId);
oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
} else {
- FeedRuntimeType runtimeType = null;
- boolean enableSubscriptionMode = false;
+ FeedRuntimeType runtimeType;
+ boolean enableSubscriptionMode;
OperatorDescriptorId opId = null;
if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
IPushRuntimeFactory[] runtimeFactories =
@@ -212,8 +209,8 @@
}
// 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();
@@ -262,12 +259,8 @@
// set absolute location constraints
for (Entry<OperatorDescriptorId, List<LocationConstraint>> entry : operatorLocations.entrySet()) {
IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
- Collections.sort(entry.getValue(), new Comparator<LocationConstraint>() {
-
- @Override
- public int compare(LocationConstraint o1, LocationConstraint o2) {
- return o1.partition - o2.partition;
- }
+ Collections.sort(entry.getValue(), (LocationConstraint o1, LocationConstraint o2) -> {
+ return o1.partition - o2.partition;
});
String[] locations = new String[entry.getValue().size()];
for (int i = 0; i < locations.length; ++i) {
@@ -321,23 +314,23 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, changingOpDesc, chosenLocations);
}
- private static IOperatorDescriptor alterJobSpecForComputeCardinality(JobSpecification spec, int requiredCardinality)
- throws AsterixException {
+ 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();
-
+ 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()) {
+ 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))) {
+ && (((FeedMetaOperatorDescriptor) targetOp).getRuntimeType()
+ .equals(FeedRuntimeType.COMPUTE))) {
connDesc = connectors.get(cid);
break;
} else {
@@ -433,7 +426,7 @@
MetadataManager.INSTANCE.acquireReadLatch();
ctx = MetadataManager.INSTANCE.beginTransaction();
feed = MetadataManager.INSTANCE.getFeed(ctx, connectionId.getFeedId().getDataverse(),
- connectionId.getFeedId().getFeedName());
+ connectionId.getFeedId().getEntityName());
preProcessingRequired = feed.getAppliedFunction() != null;
MetadataManager.INSTANCE.commitTransaction(ctx);
} catch (Exception e) {
@@ -688,7 +681,8 @@
if (function != null) {
if (function.getLanguage().equals(Function.LANGUAGE_AQL)) {
throw new NotImplementedException(
- "Secondary feeds derived from a source feed that has an applied AQL function are not supported yet.");
+ "Secondary feeds derived from a source feed that has an applied AQL function"
+ + " are not supported yet.");
} else {
outputType = function.getReturnType();
}
diff --git a/asterixdb/asterix-tools/src/main/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java b/asterixdb/asterix-tools/src/main/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
index 9514a22..a5b8f48 100644
--- a/asterixdb/asterix-tools/src/main/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
+++ b/asterixdb/asterix-tools/src/main/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
@@ -53,13 +53,13 @@
typeDataGenMap = new HashMap<TypeSignature, TypeDataGen>();
for (Statement stmt : aqlStatements) {
- if (stmt.getKind().equals(Statement.Kind.TYPE_DECL)) {
+ if (stmt.getKind() == Statement.TYPE_DECL) {
TypeDecl td = (TypeDecl) stmt;
- String typeDataverse = td.getDataverseName() == null ? defaultDataverse : td.getDataverseName()
- .getValue();
+ String typeDataverse =
+ td.getDataverseName() == null ? defaultDataverse : td.getDataverseName().getValue();
- Map<TypeSignature, IAType> typeInStmt = TypeTranslator.computeTypes(mdTxnCtx, td.getTypeDef(), td
- .getIdent().getValue(), typeDataverse, types);
+ Map<TypeSignature, IAType> typeInStmt = TypeTranslator.computeTypes(mdTxnCtx, td.getTypeDef(),
+ td.getIdent().getValue(), typeDataverse, types);
types.putAll(typeInStmt);
TypeSignature signature = new TypeSignature(typeDataverse, td.getIdent().getValue());
@@ -73,7 +73,7 @@
private String getDefaultDataverse() {
for (Statement stmt : aqlStatements) {
- if (stmt.getKind().equals(Statement.Kind.DATAVERSE_DECL)) {
+ if (stmt.getKind() == Statement.DATAVERSE_DECL) {
return ((DataverseDecl) stmt).getDataverseName().getValue();
}
}
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index d7b0fed..5c2d7c8 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -575,6 +575,7 @@
<module>asterix-replication</module>
<module>asterix-experiments</module>
<module>asterix-coverage</module>
+ <module>asterix-active</module>
</modules>
<repositories>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
index eff360b..6308636 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
@@ -21,7 +21,6 @@
import java.util.List;
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.functions.IFunctionInfo;
@@ -37,6 +36,7 @@
super(FunctionKind.SCALAR, finfo, arguments);
}
+ @SafeVarargs
public ScalarFunctionCallExpression(IFunctionInfo finfo, Mutable<ILogicalExpression>... expressions) {
super(FunctionKind.SCALAR, finfo, expressions);
}