teiid SVN: r3549 - in trunk/engine/src: main/java/org/teiid/common/buffer/impl and 1 other directories.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-12 19:23:02 -0400 (Wed, 12 Oct 2011)
New Revision: 3549
Added:
trunk/engine/src/main/java/org/teiid/common/buffer/BaseCacheEntry.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockByteBuffer.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockInputStream.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockManager.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/PartiallyOrderedCache.java
trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java
trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java
trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestPartiallyOrderedCache.java
Modified:
trunk/engine/src/main/java/org/teiid/common/buffer/CacheEntry.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
Log:
TEIID-1750 reintroducing block storage with a fixed file allocation scheme
Added: trunk/engine/src/main/java/org/teiid/common/buffer/BaseCacheEntry.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/BaseCacheEntry.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/BaseCacheEntry.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,77 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer;
+
+public class BaseCacheEntry implements Comparable<BaseCacheEntry> {
+
+ private Long id;
+ protected long lastAccess;
+ protected double orderingValue;
+
+ public BaseCacheEntry(Long id) {
+ this.id = id;
+ }
+
+ public Long getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ return getId().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return getId().toString();
+ }
+
+ public long getLastAccess() {
+ return lastAccess;
+ }
+
+ public void setLastAccess(long lastAccess) {
+ this.lastAccess = lastAccess;
+ }
+
+ public double getOrderingValue() {
+ return orderingValue;
+ }
+
+ public void setOrderingValue(double orderingValue) {
+ this.orderingValue = orderingValue;
+ }
+
+ @Override
+ public int compareTo(BaseCacheEntry o) {
+ int result = (int) Math.signum(orderingValue - o.orderingValue);
+ if (result == 0) {
+ result = Long.signum(lastAccess - o.lastAccess);
+ if (result == 0) {
+ return Long.signum(id - o.id);
+ }
+ }
+ return result;
+ }
+
+}
\ No newline at end of file
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/BaseCacheEntry.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/CacheEntry.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/CacheEntry.java 2011-10-12 20:56:57 UTC (rev 3548)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/CacheEntry.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -24,28 +24,16 @@
import java.lang.ref.WeakReference;
-public class CacheEntry implements Comparable<CacheEntry>{
+public class CacheEntry extends BaseCacheEntry {
private boolean persistent;
private Object object;
private int sizeEstimate;
- private long lastAccess;
- private double orderingValue;
private WeakReference<? extends Serializer<?>> serializer;
- private Long id;
public CacheEntry(Long id) {
- this.id = id;
+ super(id);
}
- public Long getId() {
- return id;
- }
-
- @Override
- public int hashCode() {
- return getId().hashCode();
- }
-
public int getSizeEstimate() {
return sizeEstimate;
}
@@ -53,35 +41,7 @@
public void setSizeEstimate(int sizeEstimate) {
this.sizeEstimate = sizeEstimate;
}
-
- public long getLastAccess() {
- return lastAccess;
- }
-
- public void setLastAccess(long lastAccess) {
- this.lastAccess = lastAccess;
- }
-
- public double getOrderingValue() {
- return orderingValue;
- }
-
- public void setOrderingValue(double orderingValue) {
- this.orderingValue = orderingValue;
- }
-
- @Override
- public int compareTo(CacheEntry o) {
- int result = (int) Math.signum(orderingValue - o.orderingValue);
- if (result == 0) {
- result = Long.signum(lastAccess - o.lastAccess);
- if (result == 0) {
- return Long.signum(id - o.id);
- }
- }
- return result;
- }
-
+
public boolean equals(Object obj) {
if (obj == this) {
return true;
@@ -92,10 +52,6 @@
return getId().equals(((CacheEntry)obj).getId());
}
- @Override
- public String toString() {
- return getId().toString();
- }
public Object nullOut() {
Object result = getObject();
@@ -124,8 +80,12 @@
this.serializer = serializer;
}
- public WeakReference<? extends Serializer<?>> getSerializer() {
- return serializer;
+ public Serializer<?> getSerializer() {
+ WeakReference<? extends Serializer<?>> ref = this.serializer;
+ if (ref == null) {
+ return null;
+ }
+ return ref.get();
}
}
\ No newline at end of file
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockByteBuffer.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockByteBuffer.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockByteBuffer.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,137 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Provides buffer slices or blocks off of a central
+ * set of buffers.
+ */
+public class BlockByteBuffer {
+
+ private final static class ThreadLocalByteBuffer extends ThreadLocal<ByteBuffer> {
+ private final ByteBuffer byteBuffer;
+
+ public ThreadLocalByteBuffer(ByteBuffer byteBuffer) {
+ this.byteBuffer = byteBuffer;
+ }
+
+ protected ByteBuffer initialValue() {
+ return byteBuffer.duplicate();
+ }
+ }
+
+ private static class BlockInfo {
+ final ByteBuffer bb;
+ final int block;
+ public BlockInfo(ByteBuffer bb, int block) {
+ this.bb = bb;
+ this.block = block;
+ }
+ }
+
+ private int blockAddressBits;
+ private int segmentAddressBits;
+ private int segmentSize;
+ private int blockSize;
+ private int blockCount;
+ private ThreadLocal<ByteBuffer>[] buffers;
+ private BlockInfo[] bufferCache;
+
+ /**
+ * Creates a new {@link BlockByteBuffer} where each buffer segment will be
+ * 1 << segmentAddressBits (max of 30), and a total size of (1 << blockAddressBits)*blockCount.
+ * @param segmentAddressBits
+ * @param blockCount
+ * @param blockAddressBits
+ * @param direct
+ */
+ @SuppressWarnings("unchecked")
+ public BlockByteBuffer(int segmentAddressBits, int blockCount, int blockAddressBits, boolean direct) {
+ this.segmentAddressBits = segmentAddressBits;
+ this.blockAddressBits = blockAddressBits;
+ this.blockSize = 1 << blockAddressBits;
+ this.segmentSize = 1 << this.segmentAddressBits;
+ this.blockCount = blockCount;
+ long size = ((long)blockCount)<<blockAddressBits;
+ int fullSegments = (int)size>>segmentAddressBits;
+ int lastSegmentSize = (int) (size&(segmentSize-1));
+ int segments = fullSegments;
+ if (lastSegmentSize > 0) {
+ segments++;
+ }
+ buffers = new ThreadLocal[segments];
+ for (int i = 0; i < fullSegments; i++) {
+ buffers[i] = new ThreadLocalByteBuffer(allocate(lastSegmentSize, direct));
+ }
+ if (lastSegmentSize > 0) {
+ buffers[fullSegments] = new ThreadLocalByteBuffer(allocate(lastSegmentSize, direct));
+ }
+ int logSize = 32 - Integer.numberOfLeadingZeros(blockCount);
+ bufferCache = new BlockInfo[Math.min(logSize, 20)];
+ }
+
+ public static ByteBuffer allocate(int size, boolean direct) {
+ if (direct) {
+ ByteBuffer newBuffer = ByteBuffer.allocateDirect(size);
+ int longsPerSegment = size>>3;
+ //manually initialize until java 7 when it's mandated (although this may already have been performed)
+ for (int j = 0; j < longsPerSegment; j++) {
+ newBuffer.putLong(0);
+ }
+ return newBuffer;
+ }
+ return ByteBuffer.allocate(size);
+ }
+
+ /**
+ * Return a buffer containing the given start byte.
+ * It is assumed that the caller will handle blocks in
+ * a thread safe manner.
+ * @param startIndex
+ * @return
+ */
+ public ByteBuffer getByteBuffer(int block) {
+ if (block < 0 || block >= blockCount) {
+ throw new IndexOutOfBoundsException("Invalid block " + block); //$NON-NLS-1$
+ }
+ int cacheIndex = block&(bufferCache.length -1);
+ BlockInfo info = bufferCache[cacheIndex];
+ if (info != null && info.block == block) {
+ info.bb.rewind();
+ return info.bb;
+ }
+ int segment = block>>(segmentAddressBits-blockAddressBits);
+ ByteBuffer bb = buffers[segment].get();
+ bb.limit(bb.capacity());
+ int position = (block<<blockAddressBits)&(segmentSize-1);
+ bb.position(position);
+ bb.limit(position + blockSize);
+ bb = bb.slice();
+ info = new BlockInfo(bb, block);
+ bufferCache[cacheIndex] = info;
+ return bb;
+ }
+
+}
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockByteBuffer.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockInputStream.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockInputStream.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockInputStream.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,81 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+final class BlockInputStream extends InputStream {
+ private final BlockManager manager;
+ private final int maxBlock;
+ int blockIndex;
+ ByteBuffer buf;
+ boolean free;
+ boolean done;
+
+ BlockInputStream(BlockManager manager, int blockCount, boolean free) {
+ this.manager = manager;
+ this.free = free;
+ this.maxBlock = blockCount;
+ }
+
+ @Override
+ public int read() {
+ ensureBytes();
+ if (done) {
+ return -1;
+ }
+ return buf.get() & 0xff;
+ }
+
+ private void ensureBytes() {
+ if (buf == null || buf.remaining() == 0) {
+ if (maxBlock == blockIndex) {
+ done = true;
+ if (blockIndex > 1 && free) {
+ manager.freeBlock(blockIndex - 1, false);
+ }
+ return;
+ }
+ buf = manager.getBlock(blockIndex++);
+ if (blockIndex > 2 && free) {
+ manager.freeBlock(blockIndex - 2, false);
+ }
+ }
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len) {
+ ensureBytes();
+ if (done) {
+ return -1;
+ }
+ len = Math.min(len, buf.remaining());
+ buf.get(b, off, len);
+ return len;
+ }
+
+ public int free(boolean steal) {
+ return manager.free(steal);
+ }
+}
\ No newline at end of file
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockInputStream.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockManager.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockManager.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockManager.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,48 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.nio.ByteBuffer;
+
+
+/**
+ * Represents an INode
+ */
+public interface BlockManager {
+
+ int getInode();
+
+ ByteBuffer allocateBlock(int index);
+
+ /**
+ * Get the block for a given index. Returns null if the block does not exist.
+ * @param index
+ * @return
+ */
+ ByteBuffer getBlock(int index);
+
+ int freeBlock(int index, boolean steal);
+
+ int free(boolean steal);
+
+}
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockManager.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,46 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+final class BlockOutputStream extends
+ ExtensibleBufferedOutputStream {
+ private final BlockManager blockManager;
+ int blockNum = -1;
+
+ BlockOutputStream(BlockManager blockManager) {
+ this.blockManager = blockManager;
+ }
+
+ @Override
+ protected ByteBuffer newBuffer() {
+ return blockManager.allocateBlock(++blockNum);
+ }
+
+ @Override
+ protected int flushDirect(int i) throws IOException {
+ return i;
+ }
+}
\ No newline at end of file
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,43 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import org.teiid.common.buffer.FileStore;
+import org.teiid.common.buffer.StorageManager;
+
+class BlockStore {
+ final long blockSize;
+ final ConcurrentBitSet blocksInUse;
+ final FileStore[] stores;
+
+ public BlockStore(StorageManager storageManager, int blockSize, int blockCountLog) {
+ this.blockSize = blockSize;
+ int blockCount = 1 << blockCountLog;
+ this.blocksInUse = new ConcurrentBitSet(blockCount, BufferManagerImpl.CONCURRENCY_LEVEL/2);
+ this.stores = new FileStore[BufferManagerImpl.CONCURRENCY_LEVEL/2];
+ for (int i = 0; i < stores.length; i++) {
+ this.stores[i] = storageManager.createFileStore(String.valueOf(blockSize) + '_' + i);
+ }
+ }
+
+}
\ No newline at end of file
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,714 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.teiid.common.buffer.AutoCleanupUtil;
+import org.teiid.common.buffer.BaseCacheEntry;
+import org.teiid.common.buffer.Cache;
+import org.teiid.common.buffer.CacheEntry;
+import org.teiid.common.buffer.FileStore;
+import org.teiid.common.buffer.Serializer;
+import org.teiid.common.buffer.StorageManager;
+import org.teiid.core.TeiidComponentException;
+import org.teiid.core.TeiidRuntimeException;
+import org.teiid.core.util.ObjectConverterUtil;
+import org.teiid.logging.LogConstants;
+import org.teiid.logging.LogManager;
+import org.teiid.logging.MessageLevel;
+import org.teiid.query.QueryPlugin;
+
+/**
+ * Implements storage against a {@link FileStore} abstraction using a fronting
+ * memory buffer with a filesystem paradigm. All objects must go through the
+ * memory (typically off-heap) buffer so that they can be put into their appropriately
+ * sized storage bucket.
+ *
+ * The memory uses a 31bit address space on top of 2^13 byte blocks.
+ *
+ * Therefore there is 2^31*2^13 = 2^44 or 16 terabytes max of addressable space.
+ * This is well beyond any current needs.
+ *
+ * The 64 byte inode format is:
+ * 14 32 bit direct block pointers
+ * 1 32 bit block indirect pointer
+ * 1 32 bit block doubly indirect pointer (should be rarely used)
+ *
+ * This means that the maximum number of blocks available to an object is
+ * 14 + (2^13)/4 + ((2^13)/4)^2 ~= 2^22
+ *
+ * Thus the max serialized object size is: 2^22*(2^13) ~= 32GB.
+ *
+ * Typically the max object size will be much smaller, such as 8MB.
+ *
+ * Inodes are held separately from the data/index blocks, and introduce an overhead
+ * that is ~ 1/128th the size of memory buffer.
+ *
+ * The filesystem stores are broken up into block specific sizes starting with 8KB.
+ *
+ * The root directory "physicalMapping" is held in memory for performance. It will grow in
+ * proportion to the number of tables/tuplebuffers in use.
+ *
+ * TODO: compact tail storage blocks. there may be dangling blocks causing us to consume disk space.
+ */
+public class BufferFrontedFileStoreCache implements Cache, StorageManager {
+
+ static final int ADDRESS_BITS = 31;
+ static final int SYSTEM_MASK = 1<<ADDRESS_BITS;
+ static final int BYTES_PER_BLOCK_ADDRESS = 4;
+ static final int INODE_BYTES = 16*BYTES_PER_BLOCK_ADDRESS;
+ static final int LOG_INODE_SIZE = 6;
+ static final int DIRECT_POINTERS = 14;
+ static final int EMPTY_ADDRESS = -1;
+
+ //TODO allow the block size to be configurable
+ static final int LOG_BLOCK_SIZE = 13;
+ static final int BLOCK_SIZE = 1 << LOG_BLOCK_SIZE;
+ static final int BLOCK_MASK = BLOCK_SIZE - 1;
+ static final int ADDRESSES_PER_BLOCK = BLOCK_SIZE/BYTES_PER_BLOCK_ADDRESS;
+ static final int MAX_INDIRECT = DIRECT_POINTERS + ADDRESSES_PER_BLOCK;
+ static final int MAX_DOUBLE_INDIRECT = MAX_INDIRECT + ADDRESSES_PER_BLOCK * ADDRESSES_PER_BLOCK;
+
+ private enum Mode {
+ GET,
+ UPDATE,
+ ALLOCATE
+ }
+
+ private final class InodeBlockManager implements BlockManager {
+ private int inode;
+ private ByteBuffer inodeBuffer;
+ private final long gid;
+ private final long oid;
+
+ InodeBlockManager(long gid, long oid, int inode) {
+ this.inode = inode;
+ this.gid = gid;
+ this.oid = oid;
+ }
+
+ @Override
+ public int getInode() {
+ return inode;
+ }
+
+ @Override
+ public ByteBuffer getBlock(int index) {
+ int dataBlock = getOrUpdateDataBlockIndex(index, EMPTY_ADDRESS, Mode.GET);
+ return blockByteBuffer.getByteBuffer(dataBlock);
+ }
+
+ private int getOrUpdateDataBlockIndex(int index, int value, Mode mode) {
+ if (index >= MAX_DOUBLE_INDIRECT || (mode == Mode.ALLOCATE && index >= maxMemoryBlocks)) {
+ throw new TeiidRuntimeException("Max block number exceeded"); //$NON-NLS-1$
+ }
+ int dataBlock = 0;
+ int position = 0;
+ ByteBuffer info = getInodeBlock();
+ if (index >= MAX_INDIRECT) {
+ position = BYTES_PER_BLOCK_ADDRESS*(DIRECT_POINTERS+1);
+ ByteBuffer next = updateIndirectBlockInfo(info, index, position, MAX_INDIRECT, value, mode);
+ if (next != info) {
+ info = next;
+ //should have traversed to the secondary
+ int indirectAddressBlock = (index - MAX_INDIRECT) / ADDRESSES_PER_BLOCK;
+ position = indirectAddressBlock * BYTES_PER_BLOCK_ADDRESS;
+ if (mode == Mode.ALLOCATE && position + BYTES_PER_BLOCK_ADDRESS < BLOCK_SIZE) {
+ info.putInt(position + BYTES_PER_BLOCK_ADDRESS, EMPTY_ADDRESS);
+ }
+ next = updateIndirectBlockInfo(info, index, position, MAX_INDIRECT + indirectAddressBlock * ADDRESSES_PER_BLOCK, value, mode);
+ if (next != info) {
+ info = next;
+ position = ((index - MAX_INDIRECT)%ADDRESSES_PER_BLOCK) * BYTES_PER_BLOCK_ADDRESS;
+ }
+ }
+ } else if (index >= DIRECT_POINTERS) {
+ //indirect
+ position = BYTES_PER_BLOCK_ADDRESS*DIRECT_POINTERS;
+ ByteBuffer next = updateIndirectBlockInfo(info, index, position, DIRECT_POINTERS, value, mode);
+ if (next != info) {
+ info = next;
+ position = (index - DIRECT_POINTERS) * BYTES_PER_BLOCK_ADDRESS;
+ }
+ } else {
+ position = BYTES_PER_BLOCK_ADDRESS*index;
+ }
+ if (mode == Mode.ALLOCATE) {
+ dataBlock = nextBlock(true);
+ info.putInt(position, dataBlock);
+ if (mode == Mode.ALLOCATE && position + BYTES_PER_BLOCK_ADDRESS < BLOCK_SIZE) {
+ //maintain the invariant that the next pointer is empty
+ info.putInt(position + BYTES_PER_BLOCK_ADDRESS, EMPTY_ADDRESS);
+ }
+ } else {
+ dataBlock = info.getInt(position);
+ if (mode == Mode.UPDATE) {
+ info.putInt(position, value);
+ }
+ }
+ return dataBlock;
+ }
+
+ private ByteBuffer updateIndirectBlockInfo(ByteBuffer buf, int index, int position, int cutOff, int value, Mode mode) {
+ int sib_index = buf.getInt(position);
+ if (index == cutOff) {
+ if (mode == Mode.ALLOCATE) {
+ sib_index = nextBlock(false);
+ buf.putInt(position, sib_index);
+ } else if (mode == Mode.UPDATE && value == EMPTY_ADDRESS) {
+ freeDataBlock(sib_index);
+ return buf;
+ }
+ }
+ return blockByteBuffer.getByteBuffer(sib_index);
+ }
+
+ /**
+ * Get the next dataBlock. When the memory buffer is full we have some
+ * book keeping to do.
+ * @return
+ */
+ private int nextBlock(boolean data) {
+ int next = EMPTY_ADDRESS;
+ memoryEvictionLock.readLock().lock();
+ boolean readLocked = true;
+ try {
+ if ((next = blocksInuse.getAndSetNextClearBit()) == EMPTY_ADDRESS) {
+ memoryEvictionLock.readLock().unlock();
+ readLocked = false;
+ next = evictFromMemoryBuffer();
+ }
+ } finally {
+ if (readLocked) {
+ memoryEvictionLock.readLock().unlock();
+ }
+ }
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.TRACE)) {
+ LogManager.logTrace(LogConstants.CTX_DQP, "Allocating", data?"data":"index", "block", next, "to", gid, oid); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
+ }
+ return next;
+ }
+
+ @Override
+ public int freeBlock(int index, boolean steal) {
+ int dataBlock = getOrUpdateDataBlockIndex(index, EMPTY_ADDRESS, Mode.UPDATE);
+ if (!steal) {
+ freeDataBlock(dataBlock);
+ }
+ return dataBlock;
+ }
+
+ private void freeDataBlock(int dataBlock) {
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.TRACE)) {
+ LogManager.logTrace(LogConstants.CTX_DQP, "freeing data block", dataBlock, "for", gid, oid); //$NON-NLS-1$ //$NON-NLS-2$
+ }
+ blocksInuse.clear(dataBlock);
+ }
+
+ private ByteBuffer getInodeBlock() {
+ if (inodeBuffer == null) {
+ if (inode == EMPTY_ADDRESS) {
+ this.inode = inodesInuse.getAndSetNextClearBit();
+ if (this.inode == -1) {
+ throw new AssertionError("Out of inodes"); //$NON-NLS-1$
+ }
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_DQP, "Allocating inode", this.inode, "to", gid, oid); //$NON-NLS-1$ //$NON-NLS-2$
+ }
+ ByteBuffer bb = getInodeBlock();
+ bb.putInt(EMPTY_ADDRESS);
+ }
+ inodeBuffer = inodeByteBuffer.getByteBuffer(inode);
+ }
+ return inodeBuffer;
+ }
+
+ @Override
+ public int free(boolean steal) {
+ if (this.inode == EMPTY_ADDRESS) {
+ return EMPTY_ADDRESS;
+ }
+ ByteBuffer bb = getInodeBlock();
+ int dataBlockToSteal = bb.getInt(0);
+ int indirectIndexBlock = bb.getInt(BYTES_PER_BLOCK_ADDRESS*DIRECT_POINTERS);
+ int doublyIndirectIndexBlock = bb.getInt(BYTES_PER_BLOCK_ADDRESS*(DIRECT_POINTERS+1));
+ boolean freedAll = freeBlock(steal?BYTES_PER_BLOCK_ADDRESS:0, bb, DIRECT_POINTERS, true);
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_DQP, "freeing inode", inode, "for", gid, oid); //$NON-NLS-1$ //$NON-NLS-2$
+ }
+ inodesInuse.clear(inode);
+ if (!freedAll || indirectIndexBlock == EMPTY_ADDRESS) {
+ return steal?dataBlockToSteal:EMPTY_ADDRESS;
+ }
+ freedAll = freeIndirectBlock(indirectIndexBlock);
+ if (!freedAll || doublyIndirectIndexBlock == EMPTY_ADDRESS) {
+ return steal?dataBlockToSteal:EMPTY_ADDRESS;
+ }
+ bb = blockByteBuffer.getByteBuffer(doublyIndirectIndexBlock);
+ freeBlock(0, bb, ADDRESSES_PER_BLOCK, false);
+ freeDataBlock(doublyIndirectIndexBlock);
+ return steal?dataBlockToSteal:EMPTY_ADDRESS;
+ }
+
+ private boolean freeIndirectBlock(int indirectIndexBlock) {
+ ByteBuffer bb = blockByteBuffer.getByteBuffer(indirectIndexBlock);
+ boolean freedAll = freeBlock(0, bb, ADDRESSES_PER_BLOCK, true);
+ freeDataBlock(indirectIndexBlock);
+ return freedAll;
+ }
+
+ private boolean freeBlock(int startPosition, ByteBuffer ib, int numPointers, boolean primary) {
+ ib.position(startPosition);
+ for (int i = 0; i < numPointers; i++) {
+ int dataBlock = ib.getInt();
+ if (dataBlock == EMPTY_ADDRESS) {
+ return false;
+ }
+ if (primary) {
+ freeDataBlock(dataBlock);
+ } else {
+ freeIndirectBlock(dataBlock);
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public ByteBuffer allocateBlock(int blockNum) {
+ int dataBlock = getOrUpdateDataBlockIndex(blockNum, EMPTY_ADDRESS, Mode.ALLOCATE);
+ return blockByteBuffer.getByteBuffer(dataBlock);
+ }
+ }
+
+ private static class PhysicalInfo extends BaseCacheEntry {
+ int inode = EMPTY_ADDRESS;
+ int block = EMPTY_ADDRESS;
+ int sizeIndex = 0;
+ final int memoryBlockCount;
+ final Long gid;
+
+ public PhysicalInfo(Long gid, Long id, int inode, int size) {
+ super(id);
+ this.inode = inode;
+ this.gid = gid;
+ this.memoryBlockCount = (size>>LOG_BLOCK_SIZE) + ((size&BLOCK_MASK)>0?1:0);
+ int blocks = memoryBlockCount;
+ while (blocks >= 1) {
+ this.sizeIndex++;
+ blocks>>=2;
+ }
+ }
+ }
+
+ double crfLamda = .0001;
+
+ StorageManager storageManager;
+ int maxStorageObjectSize = 1 << 23; //8MB
+ private long memoryBufferSpace = 1 << 27;
+ private boolean direct;
+
+ int maxMemoryBlocks;
+ private AtomicLong readAttempts = new AtomicLong();
+ PartiallyOrderedCache<Long, PhysicalInfo> memoryBufferEntries = new PartiallyOrderedCache<Long, PhysicalInfo>(16, .75f, BufferManagerImpl.CONCURRENCY_LEVEL) {
+
+ @Override
+ protected void recordAccess(Long key, PhysicalInfo value, boolean initial) {
+ long lastAccess = value.getLastAccess();
+ value.setLastAccess(readAttempts.get());
+ if (initial && lastAccess == 0) {
+ return;
+ }
+ double orderingValue = value.getOrderingValue();
+ orderingValue = computeNextOrderingValue(value.getLastAccess(), lastAccess, orderingValue);
+ value.setOrderingValue(orderingValue);
+ }
+
+ };
+ private Semaphore memoryWritePermits; //prevents deadlock waiting for free blocks
+ ReentrantReadWriteLock memoryEvictionLock = new ReentrantReadWriteLock();
+
+ private int blocks;
+ private ConcurrentBitSet blocksInuse;
+ private BlockByteBuffer blockByteBuffer;
+
+ private ConcurrentBitSet inodesInuse;
+ private BlockByteBuffer inodeByteBuffer;
+
+ //root directory
+ private ConcurrentHashMap<Long, Map<Long, PhysicalInfo>> physicalMapping = new ConcurrentHashMap<Long, Map<Long, PhysicalInfo>>(16, .75f, BufferManagerImpl.CONCURRENCY_LEVEL);
+ private BlockStore[] sizeBasedStores;
+
+ @Override
+ public void initialize() throws TeiidComponentException {
+ storageManager.initialize();
+ blocks = (int) Math.min(Integer.MAX_VALUE, (memoryBufferSpace>>LOG_BLOCK_SIZE));
+ inodesInuse = new ConcurrentBitSet(blocks+1, BufferManagerImpl.CONCURRENCY_LEVEL);
+ blocksInuse = new ConcurrentBitSet(blocks, BufferManagerImpl.CONCURRENCY_LEVEL);
+ this.blockByteBuffer = new BlockByteBuffer(30, blocks, LOG_BLOCK_SIZE, direct);
+ //ensure that we'll run out of blocks first
+ this.inodeByteBuffer = new BlockByteBuffer(30, blocks+1, LOG_INODE_SIZE, direct);
+ memoryBufferSpace = Math.max(memoryBufferSpace, maxStorageObjectSize);
+ memoryWritePermits = new Semaphore(Math.max(1, (int)Math.min(memoryBufferSpace/maxStorageObjectSize, Integer.MAX_VALUE)));
+ maxMemoryBlocks = Math.min(MAX_DOUBLE_INDIRECT, maxStorageObjectSize>>LOG_BLOCK_SIZE);
+ //account for index pointer block overhead
+ if (maxMemoryBlocks > DIRECT_POINTERS) {
+ maxMemoryBlocks--;
+ }
+ if (maxMemoryBlocks > MAX_INDIRECT) {
+ int indirect = maxMemoryBlocks-MAX_INDIRECT;
+ maxMemoryBlocks -= (indirect/ADDRESSES_PER_BLOCK + (indirect%ADDRESSES_PER_BLOCK>0?1:0) + 1);
+ }
+ List<BlockStore> stores = new ArrayList<BlockStore>();
+ int size = BLOCK_SIZE;
+ do {
+ stores.add(new BlockStore(this.storageManager, size, 30));
+ size <<=2;
+ } while (size>>2 < maxStorageObjectSize);
+ this.sizeBasedStores = stores.toArray(new BlockStore[stores.size()]);
+ }
+
+ double computeNextOrderingValue(long currentTime,
+ long lastAccess, double orderingValue) {
+ orderingValue =
+ //Frequency component
+ orderingValue*Math.pow(1-crfLamda, currentTime - lastAccess)
+ //recency component
+ + Math.pow(currentTime, crfLamda);
+ return orderingValue;
+ }
+
+ InodeBlockManager getBlockManager(long gid, long oid, int inode) {
+ return new InodeBlockManager(gid, oid, inode);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void add(CacheEntry entry, Serializer s) {
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_DQP, "adding object", s.getId(), entry.getId()); //$NON-NLS-1$
+ }
+ InodeBlockManager blockManager = null;
+ boolean hasPermit = false;
+ PhysicalInfo info = null;
+ boolean newEntry = true;
+ boolean success = false;
+ try {
+ Map<Long, PhysicalInfo> map = physicalMapping.get(s.getId());
+ if (map == null) {
+ return; //already removed
+ }
+ info = map.get(entry.getId());
+ if (info == null) {
+ if (!map.containsKey(entry.getId())) {
+ return; //already removed
+ }
+ } else {
+ newEntry = false;
+ synchronized (info) {
+ if (info.inode != EMPTY_ADDRESS || !shouldPlaceInMemoryBuffer(readAttempts.get(), info)) {
+ success = true;
+ return;
+ }
+ }
+ }
+ memoryWritePermits.acquire();
+ hasPermit = true;
+ blockManager = getBlockManager(s.getId(), entry.getId(), EMPTY_ADDRESS);
+ ExtensibleBufferedOutputStream fsos = new BlockOutputStream(blockManager);
+ ObjectOutputStream oos = new ObjectOutputStream(fsos);
+ oos.writeInt(entry.getSizeEstimate());
+ s.serialize(entry.getObject(), oos);
+ oos.close();
+ synchronized (map) {
+ //synchronize to ensure proper cleanup from a concurrent removal
+ if (physicalMapping.containsKey(s.getId()) && map.containsKey(entry.getId())) {
+ if (newEntry) {
+ info = new PhysicalInfo(s.getId(), entry.getId(), blockManager.getInode(), fsos.getBytesWritten());
+ map.put(entry.getId(), info);
+ memoryBufferEntries.put(entry.getId(), info);
+ }
+ success = true;
+ }
+ }
+ } catch (Throwable e) {
+ LogManager.logError(LogConstants.CTX_BUFFER_MGR, e, "Error persisting batch, attempts to read batch "+ entry.getId() +" later will result in an exception"); //$NON-NLS-1$ //$NON-NLS-2$
+ } finally {
+ if (hasPermit) {
+ memoryWritePermits.release();
+ }
+ if (!success && blockManager != null) {
+ blockManager.free(false);
+ }
+ }
+ }
+
+ @Override
+ public CacheEntry get(Long oid, Serializer<?> serializer) throws TeiidComponentException {
+ long currentTime = readAttempts.incrementAndGet();
+ try {
+ Map<Long, PhysicalInfo> map = physicalMapping.get(serializer.getId());
+ if (map == null) {
+ return null;
+ }
+ final PhysicalInfo info = map.get(oid);
+ if (info == null) {
+ return null;
+ }
+ CacheEntry ce = new CacheEntry(oid);
+ InputStream is = null;
+ synchronized (info) {
+ if (info.inode != EMPTY_ADDRESS) {
+ memoryBufferEntries.get(oid); //touch this entry
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_DQP, "Getting object at inode", info.inode, serializer.getId(), oid); //$NON-NLS-1$
+ }
+ BlockManager manager = getBlockManager(serializer.getId(), oid, info.inode);
+ is = new BlockInputStream(manager, info.memoryBlockCount, false);
+ } else if (info.block != EMPTY_ADDRESS) {
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_DQP, MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_DQP, "Getting object at block", info.block, info.sizeIndex, serializer.getId(), oid); //$NON-NLS-1$
+ }
+ BlockStore blockStore = sizeBasedStores[info.sizeIndex];
+ FileStore fs = blockStore.stores[info.block/blockStore.blocksInUse.getBitsPerSegment()];
+ long blockOffset = (info.block%blockStore.blocksInUse.getBitsPerSegment())*blockStore.blockSize;
+ is = fs.createInputStream(blockOffset);
+ if (shouldPlaceInMemoryBuffer(currentTime, info) && this.memoryWritePermits.tryAcquire()) {
+ BlockManager manager = null;
+ try {
+ manager = getBlockManager(info.gid, info.getId(), EMPTY_ADDRESS);
+ ExtensibleBufferedOutputStream os = new BlockOutputStream(manager);
+ ObjectConverterUtil.write(os, is, -1);
+ memoryBufferEntries.put(info.getId(), info);
+ is = new BlockInputStream(manager, info.memoryBlockCount, false);
+ } finally {
+ this.memoryWritePermits.release();
+ }
+ } else {
+ this.toString();
+ }
+ } else {
+ return null;
+ }
+ }
+ ObjectInputStream ois = new ObjectInputStream(is);
+ ce.setSizeEstimate(ois.readInt());
+ ce.setLastAccess(1);
+ ce.setOrderingValue(1);
+ ce.setObject(serializer.deserialize(ois));
+ ce.setPersistent(true);
+ return ce;
+ } catch(IOException e) {
+ throw new TeiidComponentException(e, QueryPlugin.Util.getString("FileStoreageManager.error_reading", oid)); //$NON-NLS-1$
+ } catch (ClassNotFoundException e) {
+ throw new TeiidComponentException(e, QueryPlugin.Util.getString("FileStoreageManager.error_reading", oid)); //$NON-NLS-1$
+ }
+ }
+
+ private boolean shouldPlaceInMemoryBuffer(long currentTime, PhysicalInfo info) {
+ Map.Entry<PhysicalInfo, Long> lowest = memoryBufferEntries.firstEntry();
+ return lowest == null
+ || (blocksInuse.getTotalBits() - blocksInuse.getBitsSet()) > (info.memoryBlockCount<<3)
+ || lowest.getKey().getOrderingValue() < computeNextOrderingValue(currentTime, info.getLastAccess(), info.getOrderingValue());
+ }
+
+ @Override
+ public FileStore createFileStore(String name) {
+ return storageManager.createFileStore(name);
+ }
+
+ public void setDirect(boolean direct) {
+ this.direct = direct;
+ }
+
+ @Override
+ public void addToCacheGroup(Long gid, Long oid) {
+ Map<Long, PhysicalInfo> map = physicalMapping.get(gid);
+ if (map == null) {
+ return;
+ }
+ map.put(oid, null);
+ }
+
+ @Override
+ public void createCacheGroup(Long gid) {
+ physicalMapping.put(gid, Collections.synchronizedMap(new HashMap<Long, PhysicalInfo>()));
+ }
+
+ @Override
+ public void remove(Long gid, Long id) {
+ Map<Long, PhysicalInfo> map = physicalMapping.get(gid);
+ if (map == null) {
+ return;
+ }
+ PhysicalInfo info = map.remove(id);
+ free(id, info, false);
+ }
+
+ @Override
+ public Collection<Long> removeCacheGroup(Long gid) {
+ Map<Long, PhysicalInfo> map = physicalMapping.remove(gid);
+ if (map == null) {
+ return Collections.emptySet();
+ }
+ synchronized (map) {
+ for (Map.Entry<Long, PhysicalInfo> entry : map.entrySet()) {
+ free(entry.getKey(), entry.getValue(), false);
+ }
+ return map.keySet();
+ }
+ }
+
+ int free(Long oid, PhysicalInfo info, boolean demote) {
+ memoryBufferEntries.remove(oid);
+ if (info == null) {
+ return EMPTY_ADDRESS;
+ }
+ synchronized (info) {
+ memoryBufferEntries.remove(oid);
+ if (info.inode == EMPTY_ADDRESS) {
+ return EMPTY_ADDRESS;
+ }
+ BlockManager bm = getBlockManager(info.gid, oid, info.inode);
+ info.inode = EMPTY_ADDRESS;
+ if (demote) {
+ if (info.block == EMPTY_ADDRESS) {
+ BlockInputStream is = new BlockInputStream(bm, info.memoryBlockCount, true);
+ BlockStore blockStore = sizeBasedStores[info.sizeIndex];
+ FileStore fs = blockStore.stores[info.block/blockStore.blocksInUse.getBitsPerSegment()];
+ info.block = getAndSetNextClearBit(blockStore);
+ long blockOffset = (info.block%blockStore.blocksInUse.getBitsPerSegment())*blockStore.blockSize;
+ byte[] b = new byte[BLOCK_SIZE];
+ int read = 0;
+ boolean errored = false;
+ while ((read = is.read(b, 0, b.length)) != -1) {
+ if (!errored) {
+ try {
+ fs.write(blockOffset, b, 0, read);
+ blockOffset+=read;
+ } catch (Throwable e) {
+ //just continue to free
+ errored = true;
+ LogManager.logError(LogConstants.CTX_DQP, e, "Error transferring block to storage " + oid); //$NON-NLS-1$
+ }
+ }
+ }
+ return is.free(true);
+ }
+ return bm.free(true);
+ }
+ bm.free(false);
+ if (info.block != EMPTY_ADDRESS) {
+ BlockStore blockStore = sizeBasedStores[info.sizeIndex];
+ blockStore.blocksInUse.clear(info.block);
+ info.block = EMPTY_ADDRESS;
+ }
+ }
+ return EMPTY_ADDRESS;
+ }
+
+ static int getAndSetNextClearBit(BlockStore bs) {
+ int result = bs.blocksInUse.getAndSetNextClearBit();
+ if (result == -1) {
+ throw new TeiidRuntimeException("Out of blocks of size " + bs.blockSize); //$NON-NLS-1$
+ }
+ return result;
+ }
+
+ /**
+ * Stop the world eviction. Hopefully this should rarely happen.
+ * @return the stole dataBlock
+ */
+ int evictFromMemoryBuffer() {
+ memoryEvictionLock.writeLock().lock();
+ int next = -1;
+ boolean writeLocked = true;
+ try {
+ for (int i = 0; i < 10 && next == EMPTY_ADDRESS; i++) {
+ AutoCleanupUtil.doCleanup();
+ Iterator<Map.Entry<PhysicalInfo, Long>> iter = memoryBufferEntries.getEvictionQueue().entrySet().iterator();
+ while ((next = blocksInuse.getAndSetNextClearBit()) == EMPTY_ADDRESS && iter.hasNext()) {
+ Map.Entry<PhysicalInfo, Long> entry = iter.next();
+ PhysicalInfo info = entry.getKey();
+ synchronized (info) {
+ if (info.inode == EMPTY_ADDRESS) {
+ continue;
+ }
+ memoryEvictionLock.writeLock().unlock();
+ writeLocked = false;
+ next = free(entry.getValue(), info, true);
+ }
+ break;
+ }
+ }
+ if (next == -1) {
+ throw new AssertionError("Could not free space for pending write"); //$NON-NLS-1$
+ }
+ } finally {
+ if (writeLocked) {
+ memoryEvictionLock.writeLock().unlock();
+ }
+ }
+ return next;
+ }
+
+ public void setStorageManager(StorageManager storageManager) {
+ this.storageManager = storageManager;
+ }
+
+ public StorageManager getStorageManager() {
+ return storageManager;
+ }
+
+ public void setMemoryBufferSpace(long maxBufferSpace) {
+ this.memoryBufferSpace = Math.min(maxBufferSpace, 1l<<(ADDRESS_BITS+LOG_BLOCK_SIZE));
+ }
+
+ public int getInodesInUse() {
+ return this.inodesInuse.getBitsSet();
+ }
+
+ public int getDataBlocksInUse() {
+ return this.blocksInuse.getBitsSet();
+ }
+
+ public void setMaxStorageObjectSize(int maxStorageBlockSize) {
+ this.maxStorageObjectSize = maxStorageBlockSize;
+ }
+
+}
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-10-12 20:56:57 UTC (rev 3548)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -187,7 +187,7 @@
try {
//it's expected that the containing structure has updated the lob manager
BatchSerializer.writeBatch(oos, types, obj);
- } catch (IndexOutOfBoundsException e) {
+ } catch (RuntimeException e) {
//there is a chance of a concurrent persist while modifying
//in which case we want to swallow this exception
if (list == null || list.getModCount() == expectedModCount) {
@@ -219,7 +219,7 @@
return (List<List<?>>)(!retain?ce.nullOut():ce.getObject());
}
long count = readCount.incrementAndGet();
- if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.TRACE)) {
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, id, id, "reading batch", batch, "from storage, total reads:", count); //$NON-NLS-1$ //$NON-NLS-2$
}
ce = cache.get(batch, this);
@@ -268,6 +268,7 @@
}
}
+ static final int CONCURRENCY_LEVEL = 32; //TODO: make this configurable
private static final int TARGET_BYTES_PER_ROW = 1 << 11; //2k bytes per row
private static ReferenceQueue<CacheEntry> SOFT_QUEUE = new ReferenceQueue<CacheEntry>();
@@ -292,11 +293,11 @@
//combined recency/frequency lamda value between 0 and 1 lower -> LFU, higher -> LRU
//TODO: adaptively adjust this value. more hits should move closer to lru
- private final double crfLamda = .0002;
+ private final double crfLamda = .001;
//implements a LRFU cache using the a customized crf function. we store the value with
//the cache entry to make a better decision about reuse of the batch
//TODO: consider the size estimate in the weighting function
- private OrderedCache<Long, CacheEntry> memoryEntries = new OrderedCache<Long, CacheEntry>() {
+ private PartiallyOrderedCache<Long, CacheEntry> memoryEntries = new PartiallyOrderedCache<Long, CacheEntry>(16, .75f, CONCURRENCY_LEVEL) {
@Override
protected void recordAccess(Long key, CacheEntry value, boolean initial) {
@@ -599,7 +600,7 @@
}
void evict(CacheEntry ce) throws Exception {
- Serializer<?> s = ce.getSerializer().get();
+ Serializer<?> s = ce.getSerializer();
if (s == null) {
return;
}
@@ -615,8 +616,8 @@
if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, ce.getId(), "writing batch to storage, total writes: ", count); //$NON-NLS-1$
}
- cache.add(ce, s);
}
+ cache.add(ce, s);
if (s.useSoftCache()) {
createSoftReference(ce);
} else if (useWeakReferences) {
@@ -689,7 +690,7 @@
if (inMemory) {
activeBatchKB.addAndGet(-ce.getSizeEstimate());
}
- Serializer<?> s = ce.getSerializer().get();
+ Serializer<?> s = ce.getSerializer();
if (s != null) {
cache.remove(s.getId(), ce.getId());
}
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,145 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.util.BitSet;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A segmented {@link BitSet} that supports greater concurrency
+ * and faster finding of clear bits.
+ */
+public class ConcurrentBitSet {
+
+ private static class Segment {
+ int offset;
+ int maxBits;
+ int startSearch;
+ int bitsSet;
+ final private BitSet bitSet;
+
+ public Segment(int bitCount) {
+ bitSet = new BitSet();
+ maxBits = bitCount;
+ }
+ }
+
+ private int bitsPerSegment;
+ private int totalBits;
+ private AtomicInteger counter = new AtomicInteger();
+ private AtomicInteger bitsSet = new AtomicInteger();
+ private Segment[] segments;
+
+ public ConcurrentBitSet(int maxBits, int concurrencyLevel) {
+ if (maxBits < concurrencyLevel) {
+ concurrencyLevel = 1;
+ while (maxBits > 2*concurrencyLevel) {
+ concurrencyLevel <<=1;
+ }
+ }
+ segments = new Segment[concurrencyLevel];
+ bitsPerSegment = maxBits/concurrencyLevel;
+ int modBits = maxBits%concurrencyLevel;
+ if (modBits > 0) {
+ bitsPerSegment++;
+ }
+ for (int i = 0; i < concurrencyLevel; i++) {
+ segments[i] = new Segment(bitsPerSegment);
+ segments[i].offset = i*bitsPerSegment;
+ if (i == concurrencyLevel - 1) {
+ segments[i].maxBits -= (bitsPerSegment * concurrencyLevel)-maxBits;
+ }
+ }
+ this.totalBits = maxBits;
+ }
+
+ public void clear(int bitIndex) {
+ checkIndex(bitIndex);
+ Segment s = segments[bitIndex/bitsPerSegment];
+ bitIndex = bitIndex%bitsPerSegment;
+ synchronized (s) {
+ if (!s.bitSet.get(bitIndex)) {
+ throw new AssertionError(bitIndex + " not set"); //$NON-NLS-1$
+ }
+ s.bitSet.clear(bitIndex);
+ s.bitsSet--;
+ }
+ bitsSet.decrementAndGet();
+ }
+
+ /**
+ * Makes a best effort to atomically find the next clear bit and set it
+ * @return the next bit index or -1 if no clear bits are founds
+ */
+ public int getAndSetNextClearBit() {
+ int start = counter.getAndIncrement();
+ int nextBit = -1;
+ for (int i = 0; i < segments.length; i++) {
+ Segment s = segments[(start+i)&(segments.length-1)];
+ synchronized (s) {
+ if (s.bitsSet == s.maxBits) {
+ continue;
+ }
+ nextBit = s.bitSet.nextClearBit(s.startSearch);
+ if (nextBit >= s.maxBits - 1) {
+ s.startSearch = 0;
+ nextBit = s.bitSet.nextClearBit(s.startSearch);
+ if (nextBit >= s.maxBits) {
+ throw new AssertionError("could not find clear bit"); //$NON-NLS-1$
+ }
+ }
+ s.bitsSet++;
+ s.bitSet.set(nextBit);
+ s.startSearch = nextBit + 1;
+ if (s.startSearch == s.maxBits) {
+ s.startSearch = 0;
+ }
+ nextBit += s.offset;
+ break;
+ }
+ }
+ if (nextBit != -1) {
+ bitsSet.getAndIncrement();
+ }
+ return nextBit;
+ }
+
+ private void checkIndex(int bitIndex) {
+ if (bitIndex >= totalBits) {
+ throw new ArrayIndexOutOfBoundsException(bitIndex);
+ }
+ }
+
+ public int getTotalBits() {
+ return totalBits;
+ }
+
+ public int getBitsSet() {
+ return bitsSet.get();
+ }
+
+ public int getBitsPerSegment() {
+ return bitsPerSegment;
+ }
+
+}
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/main/java/org/teiid/common/buffer/impl/PartiallyOrderedCache.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/PartiallyOrderedCache.java (rev 0)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/PartiallyOrderedCache.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,149 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+public abstract class PartiallyOrderedCache<K, V> {
+
+ private int maxOrderedSize = 1 << 19;
+
+ protected Map<K, V> map;
+ //TODO: until Java 7 ConcurrentSkipListMap has a scaling bug in that
+ //the level limits the effective map size to ~ 2^16
+ //where it performs comparably under load to a synchronized LinkedHashMap
+ //just with more CPU overhead vs. wait time.
+ //TODO: have the concurrent version be pluggable
+ protected NavigableMap<V, K> evictionQueue = new TreeMap<V, K>();
+ //when we get to extreme number of entries we overflow into lru
+ protected Map<V, K> evictionQueueHead = new LinkedHashMap<V, K>();
+ //holds entries that are being evicted, but that might not yet be in a lower caching level
+ protected Map<K, V> limbo;
+
+ public PartiallyOrderedCache(int initialCapacity, float loadFactor, int concurrencyLevel) {
+ map = new ConcurrentHashMap<K, V>(initialCapacity, loadFactor, concurrencyLevel);
+ limbo = new ConcurrentHashMap<K, V>(initialCapacity, loadFactor, concurrencyLevel);
+ }
+
+ public void setMaxOrderedSize(int maxOrderedSize) {
+ this.maxOrderedSize = maxOrderedSize;
+ }
+
+ public V get(K key) {
+ V result = map.get(key);
+ if (result == null) {
+ result = limbo.get(key);
+ }
+ if (result != null) {
+ maintainQueues(key, result, null);
+ }
+ return result;
+ }
+
+ public V remove(K key) {
+ V result = map.remove(key);
+ if (result != null) {
+ synchronized (this) {
+ if (evictionQueue.remove(result) != null) {
+ orderedRemoved();
+ } else {
+ evictionQueueHead.remove(result);
+ }
+ }
+ }
+ return result;
+ }
+
+ private void orderedRemoved() {
+ if (evictionQueue.size() < (maxOrderedSize>>1) && evictionQueueHead.size() > 0) {
+ Iterator<Map.Entry<V,K>> i = evictionQueueHead.entrySet().iterator();
+ if (i.hasNext()) {
+ Map.Entry<V, K> entry = i.next();
+ if (map.containsKey(entry.getValue())) {
+ i.remove();
+ evictionQueue.put(entry.getKey(), entry.getValue());
+ }
+ }
+ }
+ }
+
+ public V put(K key, V value) {
+ V result = map.put(key, value);
+ maintainQueues(key, value, result);
+ return result;
+ }
+
+ private void maintainQueues(K key, V value, V old) {
+ synchronized (this) {
+ if (old != null && evictionQueue.remove(old) == null) {
+ evictionQueueHead.remove(old);
+ }
+ recordAccess(key, value, old == null);
+ evictionQueue.put(value, key);
+ if (evictionQueue.size() > maxOrderedSize) {
+ Map.Entry<V, K> last = evictionQueue.pollLastEntry();
+ if (last != null) {
+ if (map.containsKey(last.getValue()) && !evictionQueue.containsKey(last.getKey())) {
+ evictionQueueHead.put(last.getKey(), last.getValue());
+ }
+ }
+ }
+ }
+ }
+
+ public V evict() {
+ Map.Entry<V, K> entry = evictionQueue.pollFirstEntry();
+ if (entry == null) {
+ return null;
+ }
+ synchronized (this) {
+ orderedRemoved();
+ }
+ limbo.put(entry.getValue(), entry.getKey());
+ return map.remove(entry.getValue());
+ }
+
+ public Map<V, K> getEvictionQueue() {
+ return evictionQueue;
+ }
+
+ public Map.Entry<V, K> firstEntry() {
+ return evictionQueue.firstEntry();
+ }
+
+ public void finishedEviction(K key) {
+ limbo.remove(key);
+ }
+
+ public int size() {
+ return map.size();
+ }
+
+ protected abstract void recordAccess(K key, V value, boolean initial);
+
+}
Property changes on: trunk/engine/src/main/java/org/teiid/common/buffer/impl/PartiallyOrderedCache.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java (rev 0)
+++ trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,171 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.lang.ref.WeakReference;
+
+import org.junit.Test;
+import org.teiid.common.buffer.CacheEntry;
+import org.teiid.common.buffer.Serializer;
+import org.teiid.core.TeiidComponentException;
+
+public class TestBufferFrontedFileStoreCache {
+
+ private final class SimpleSerializer implements Serializer<Integer> {
+ @Override
+ public Integer deserialize(ObjectInputStream ois)
+ throws IOException, ClassNotFoundException {
+ Integer result = ois.readInt();
+ for (int i = 0; i < result; i++) {
+ assertEquals(i, ois.readInt());
+ }
+ return result;
+ }
+
+ @Override
+ public Long getId() {
+ return 1l;
+ }
+
+ @Override
+ public void serialize(Integer obj, ObjectOutputStream oos)
+ throws IOException {
+ oos.writeInt(obj);
+ for (int i = 0; i < obj; i++) {
+ oos.writeInt(i);
+ }
+ }
+
+ @Override
+ public boolean useSoftCache() {
+ return false;
+ }
+ }
+
+ @Test public void testAddGetMultiBlock() throws Exception {
+ BufferFrontedFileStoreCache cache = createLayeredCache(1 << 26, 1 << 26);
+
+ CacheEntry ce = new CacheEntry(2l);
+ Serializer<Integer> s = new SimpleSerializer();
+ cache.createCacheGroup(s.getId());
+ Integer cacheObject = Integer.valueOf(2);
+ ce.setObject(cacheObject);
+ cache.addToCacheGroup(s.getId(), ce.getId());
+ cache.add(ce, s);
+
+ ce = cache.get(2l, s);
+ assertEquals(cacheObject, ce.getObject());
+
+ //test something that exceeds the direct inode data blocks
+ ce = new CacheEntry(3l);
+ cacheObject = Integer.valueOf(80000);
+ ce.setObject(cacheObject);
+ cache.addToCacheGroup(s.getId(), ce.getId());
+ cache.add(ce, s);
+
+ ce = cache.get(3l, s);
+ assertEquals(cacheObject, ce.getObject());
+
+ cache.removeCacheGroup(1l);
+
+ assertEquals(0, cache.getDataBlocksInUse());
+ assertEquals(0, cache.getInodesInUse());
+
+ //test something that exceeds the indirect data blocks
+ ce = new CacheEntry(3l);
+ cache.createCacheGroup(s.getId());
+ cacheObject = Integer.valueOf(5000000);
+ ce.setObject(cacheObject);
+ cache.addToCacheGroup(s.getId(), ce.getId());
+ cache.add(ce, s);
+
+ ce = cache.get(3l, s);
+ assertEquals(cacheObject, ce.getObject());
+
+ cache.removeCacheGroup(1l);
+
+ assertEquals(0, cache.getDataBlocksInUse());
+ assertEquals(0, cache.getInodesInUse());
+
+ //test something that exceeds the allowable object size
+ ce = new CacheEntry(3l);
+ cache.createCacheGroup(s.getId());
+ cacheObject = Integer.valueOf(500000000);
+ ce.setObject(cacheObject);
+ cache.addToCacheGroup(s.getId(), ce.getId());
+ cache.add(ce, s);
+
+ ce = cache.get(3l, s);
+ assertNull(ce);
+
+ cache.removeCacheGroup(1l);
+
+ assertEquals(0, cache.getDataBlocksInUse());
+ assertEquals(0, cache.getInodesInUse());
+ }
+
+ @Test public void testEviction() throws Exception {
+ BufferFrontedFileStoreCache cache = createLayeredCache(1<<15, 1<<15);
+
+ CacheEntry ce = new CacheEntry(2l);
+ Serializer<Integer> s = new SimpleSerializer();
+ WeakReference<? extends Serializer<?>> ref = new WeakReference<Serializer<?>>(s);
+ ce.setSerializer(ref);
+ cache.createCacheGroup(s.getId());
+ Integer cacheObject = Integer.valueOf(5000);
+ ce.setObject(cacheObject);
+ cache.addToCacheGroup(s.getId(), ce.getId());
+ cache.add(ce, s);
+
+ ce = new CacheEntry(3l);
+ ce.setSerializer(ref);
+ cacheObject = Integer.valueOf(5000);
+ ce.setObject(cacheObject);
+ cache.addToCacheGroup(s.getId(), ce.getId());
+ cache.add(ce, s);
+
+ assertEquals(3, cache.getDataBlocksInUse());
+ assertEquals(1, cache.getInodesInUse());
+
+ ce = cache.get(2l, s);
+ assertEquals(Integer.valueOf(5000), ce.getObject());
+ }
+
+ private BufferFrontedFileStoreCache createLayeredCache(int bufferSpace, int objectSize) throws TeiidComponentException {
+ BufferFrontedFileStoreCache fsc = new BufferFrontedFileStoreCache();
+ fsc.setMemoryBufferSpace(bufferSpace);
+ fsc.setMaxStorageObjectSize(objectSize);
+ fsc.setDirect(false);
+ SplittableStorageManager ssm = new SplittableStorageManager(new MemoryStorageManager());
+ ssm.setMaxFileSizeDirect(MemoryStorageManager.MAX_FILE_SIZE);
+ fsc.setStorageManager(ssm);
+ fsc.initialize();
+ return fsc;
+ }
+
+}
Property changes on: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java (rev 0)
+++ trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,65 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+ package org.teiid.common.buffer.impl;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+public class TestConcurrentBitSet {
+
+ @Test public void testBitsSet() {
+ ConcurrentBitSet bst = new ConcurrentBitSet(50001, 4);
+ assertEquals(0, bst.getAndSetNextClearBit());
+ assertEquals(12501, bst.getAndSetNextClearBit());
+ assertEquals(25002, bst.getAndSetNextClearBit());
+ assertEquals(37503, bst.getAndSetNextClearBit());
+ assertEquals(1, bst.getAndSetNextClearBit());
+ assertEquals(5, bst.getBitsSet());
+ bst.clear(1);
+ assertEquals(4, bst.getBitsSet());
+ bst.clear(12501);
+ try {
+ bst.clear(30000);
+ fail();
+ } catch (AssertionError e) {
+
+ }
+ assertEquals(3, bst.getBitsSet());
+
+ for (int i = 0; i < bst.getTotalBits()-3;i++) {
+ assertTrue(bst.getAndSetNextClearBit() != -1);
+ }
+
+ bst.clear(5);
+ bst.clear(12505);
+ bst.clear(25505);
+ bst.clear(37505);
+
+ for (int i = 0; i < 4; i++) {
+ int bit = bst.getAndSetNextClearBit();
+ assertTrue(bit < bst.getTotalBits() && bit > 0);
+ }
+ }
+
+}
Property changes on: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestPartiallyOrderedCache.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestPartiallyOrderedCache.java (rev 0)
+++ trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestPartiallyOrderedCache.java 2011-10-12 23:23:02 UTC (rev 3549)
@@ -0,0 +1,61 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+
+public class TestPartiallyOrderedCache {
+
+ @Test public void testQueueMaintenance() {
+ PartiallyOrderedCache<Integer, Integer> cache = new PartiallyOrderedCache<Integer, Integer>(16, .75f, 16) {
+
+ @Override
+ protected void recordAccess(Integer key, Integer value, boolean initial) {
+
+ }
+ };
+
+ cache.setMaxOrderedSize(5);
+
+ for (int i = 0; i < 10; i++) {
+ cache.put(i, i);
+ }
+
+ cache.get(8);
+ cache.get(1);
+
+ List<Integer> evictions = new ArrayList<Integer>();
+ for (int i = 0; i < 10; i++) {
+ evictions.add(i);
+ }
+ //we expect natural order because the lru is converted into the sorted on natural key
+ assertEquals(Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9), evictions);
+ }
+
+}
Property changes on: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestPartiallyOrderedCache.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
13 years, 2 months
teiid SVN: r3548 - in branches/as7: build/kits/jboss-as7/standalone/configuration and 11 other directories.
by teiid-commits@lists.jboss.org
Author: rareddy
Date: 2011-10-12 16:56:57 -0400 (Wed, 12 Oct 2011)
New Revision: 3548
Added:
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/ClientServiceRegistryReferenceFactoryService.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Transport.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportAdd.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportRemove.java
Removed:
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Configuration.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineAdd.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineReferenceFactoryService.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineRemove.java
branches/as7/jboss-integration/src/test/java/org/teiid/jboss/MockXMLExtendedWriter.java
branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidConfiguration.java
Modified:
branches/as7/build/kits/jboss-as7/domain/configuration/domain-teiid.xml
branches/as7/build/kits/jboss-as7/standalone/configuration/standalone-teiid.xml
branches/as7/client/src/main/java/org/teiid/net/TeiidURL.java
branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java
branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Element.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineOperationHandler.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidBootServicesAdd.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidExtension.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidServiceNames.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemDescribe.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemParser.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TranslatorAdd.java
branches/as7/jboss-integration/src/main/java/org/teiid/jboss/deployers/RuntimeEngineDeployer.java
branches/as7/jboss-integration/src/main/resources/org/teiid/jboss/i18n.properties
branches/as7/jboss-integration/src/main/resources/schema/jboss-teiid.xsd
branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidAdminOperations.java
branches/as7/jboss-integration/src/test/resources/teiid-model-config.txt
branches/as7/jboss-integration/src/test/resources/teiid-model-json.txt
branches/as7/jboss-integration/src/test/resources/teiid-sample-config.xml
branches/as7/runtime/src/main/java/org/teiid/transport/LocalServerConnection.java
branches/as7/runtime/src/main/java/org/teiid/transport/ODBCSocketListener.java
branches/as7/runtime/src/main/java/org/teiid/transport/SocketConfiguration.java
branches/as7/test-integration/common/src/test/java/org/teiid/transport/TestODBCSocketTransport.java
branches/as7/test-integration/db/pom.xml
Log:
TEIID-1720: Removed engine as submodel, added transport as submodel. Many changes to how the model nodes are defined and work
Modified: branches/as7/build/kits/jboss-as7/domain/configuration/domain-teiid.xml
===================================================================
--- branches/as7/build/kits/jboss-as7/domain/configuration/domain-teiid.xml 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/build/kits/jboss-as7/domain/configuration/domain-teiid.xml 2011-10-12 20:56:57 UTC (rev 3548)
@@ -323,15 +323,16 @@
</security-domains>
</subsystem>
<subsystem xmlns="urn:jboss:domain:teiid:1.0">
- <async-thread-group>teiid-async</async-thread-group>
+ <async-thread-pool>teiid-async</async-thread-pool>
<object-replicator stack="udp" cluster-name="teiid-rep"/>
<resultset-cache container-name="teiid" name="resultset"/>
- <query-engine name = "default">
- <security-domain>teiid-security</security-domain>
- <jdbc socket-binding="teiid-jdbc"/>
- <odbc socket-binding="teiid-odbc"/>
- </query-engine>
+ <transport name="jdbc" protocol="teiid" socket-binding="teiid-jdbc">
+ <authentication security-domain="teiid-security" />
+ </transport>
+ <transport name="odbc" protocol="pg" socket-binding="teiid-odbc">
+ <authentication security-domain="teiid-security" />
+ </transport>
<translator name="jdbc-simple" module="org.jboss.teiid.translator.jdbc"/>
<translator name="jdbc-ansi" module="org.jboss.teiid.translator.jdbc"/>
Modified: branches/as7/build/kits/jboss-as7/standalone/configuration/standalone-teiid.xml
===================================================================
--- branches/as7/build/kits/jboss-as7/standalone/configuration/standalone-teiid.xml 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/build/kits/jboss-as7/standalone/configuration/standalone-teiid.xml 2011-10-12 20:56:57 UTC (rev 3548)
@@ -136,6 +136,9 @@
<driver name="h2" module="com.h2database.h2">
<xa-datasource-class>org.h2.jdbcx.JdbcDataSource</xa-datasource-class>
</driver>
+ <driver name="mysql" module="com.mysql">
+ <xa-datasource-class>com.mysql.jdbc.jdbc2.optional.MysqlXADataSource</xa-datasource-class>
+ </driver>
<driver name="oracle" module="com.oracle">
<xa-datasource-class>oracle.jdbc.xa.client.OracleXADataSource</xa-datasource-class>
</driver>
@@ -313,15 +316,17 @@
</security-domains>
</subsystem>
<subsystem xmlns="urn:jboss:domain:teiid:1.0">
- <async-thread-group>teiid-async</async-thread-group>
+ <async-thread-pool>teiid-async</async-thread-pool>
<resultset-cache container-name="teiid" name="resultset"/>
- <query-engine name = "default">
- <security-domain>teiid-security</security-domain>
- <jdbc socket-binding="teiid-jdbc"/>
- <odbc socket-binding="teiid-odbc"/>
- </query-engine>
-
+ <transport name="embedded"/>
+ <transport name="jdbc" protocol="teiid" socket-binding="teiid-jdbc">
+ <authentication security-domain="teiid-security"/>
+ </transport>
+ <transport name="odbc" protocol="pg" socket-binding="teiid-odbc">
+ <authentication security-domain="teiid-security"/>
+ </transport>
+
<translator name="jdbc-simple" module="org.jboss.teiid.translator.jdbc"/>
<translator name="jdbc-ansi" module="org.jboss.teiid.translator.jdbc"/>
<translator name="access" module="org.jboss.teiid.translator.jdbc"/>
Modified: branches/as7/client/src/main/java/org/teiid/net/TeiidURL.java
===================================================================
--- branches/as7/client/src/main/java/org/teiid/net/TeiidURL.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/client/src/main/java/org/teiid/net/TeiidURL.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -90,8 +90,6 @@
public enum AuthenticationType {
CLEARTEXT,KRB5
};
-
- public static final String ENGINE_NAME = "engineName"; //$NON-NLS-1$
}
public static final String DOT_DELIMITER = "."; //$NON-NLS-1$
Modified: branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java
===================================================================
--- branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPConfiguration.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -45,7 +45,7 @@
private boolean exceptionOnMaxSourceRows = true;
private int maxSourceRows = -1;
private int maxActivePlans = DEFAULT_MAX_ACTIVE_PLANS;
- private int maxODBCLobSizeAllowed = 5*1024*1024; // 5 MB
+
private int userRequestSourceConcurrency = DEFAULT_USER_REQUEST_SOURCE_CONCURRENCY;
private boolean detectingChangeEvents = true;
@@ -144,15 +144,7 @@
public void setMaxSourceRows(int maxSourceRows) {
this.maxSourceRows = maxSourceRows;
}
-
- public int getMaxODBCLobSizeAllowed() {
- return this.maxODBCLobSizeAllowed;
- }
-
- public void setMaxODBCLobSizeAllowed(int lobSize) {
- this.maxODBCLobSizeAllowed = lobSize;
- }
-
+
public AuthorizationValidator getAuthorizationValidator() {
return authorizationValidator;
}
Modified: branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java
===================================================================
--- branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/engine/src/main/java/org/teiid/dqp/internal/process/DQPCore.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -22,7 +22,12 @@
package org.teiid.dqp.internal.process;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.FutureTask;
@@ -31,11 +36,9 @@
import javax.resource.spi.work.Work;
import javax.transaction.xa.Xid;
-import org.teiid.adminapi.Admin;
import org.teiid.adminapi.AdminException;
import org.teiid.adminapi.Request.ProcessingState;
import org.teiid.adminapi.Request.ThreadState;
-import org.teiid.adminapi.impl.CacheStatisticsMetadata;
import org.teiid.adminapi.impl.RequestMetadata;
import org.teiid.adminapi.impl.TransactionMetadata;
import org.teiid.adminapi.impl.WorkerPoolStatisticsMetadata;
@@ -53,6 +56,7 @@
import org.teiid.core.TeiidProcessingException;
import org.teiid.core.TeiidRuntimeException;
import org.teiid.core.types.Streamable;
+import org.teiid.core.util.ApplicationInfo;
import org.teiid.dqp.internal.process.ThreadReuseExecutor.PrioritizedRunnable;
import org.teiid.dqp.message.AtomicRequestMessage;
import org.teiid.dqp.message.RequestID;
@@ -61,8 +65,11 @@
import org.teiid.dqp.service.TransactionContext.Scope;
import org.teiid.dqp.service.TransactionService;
import org.teiid.events.EventDistributor;
-import org.teiid.logging.*;
+import org.teiid.logging.CommandLogMessage;
import org.teiid.logging.CommandLogMessage.Event;
+import org.teiid.logging.LogConstants;
+import org.teiid.logging.LogManager;
+import org.teiid.logging.MessageLevel;
import org.teiid.metadata.MetadataRepository;
import org.teiid.query.QueryPlugin;
import org.teiid.query.tempdata.TempTableDataManager;
@@ -801,6 +808,10 @@
return this.config.getMaxSourceRows();
}
+ public int getMaxRowsFetchSize() {
+ return this.config.getMaxRowsFetchSize();
+ }
+
public void setResultsetCache(SessionAwareCache<CachedResults> cache) {
this.rsCache = cache;
}
@@ -824,4 +835,8 @@
SessionAwareCache<PreparedPlan> getPrepPlanCache() {
return prepPlanCache;
}
+
+ public String getRuntimeVersion() {
+ return ApplicationInfo.getInstance().getBuildNumber();
+ }
}
\ No newline at end of file
Copied: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/ClientServiceRegistryReferenceFactoryService.java (from rev 3506, branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineReferenceFactoryService.java)
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/ClientServiceRegistryReferenceFactoryService.java (rev 0)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/ClientServiceRegistryReferenceFactoryService.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -0,0 +1,62 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * Copyright 2011, Red Hat, Inc., and individual contributors
+ * as indicated by the @author tags. See the copyright.txt file in the
+ * distribution for a full listing of individual contributors.
+ *
+ * This is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU Lesser General Public License as
+ * published by the Free Software Foundation; either version 2.1 of
+ * the License, or (at your option) any later version.
+ *
+ * This software is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this software; if not, write to the Free
+ * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
+ * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
+ */
+
+package org.teiid.jboss;
+
+import org.jboss.as.naming.ManagedReference;
+import org.jboss.as.naming.ManagedReferenceFactory;
+import org.jboss.as.naming.ValueManagedReference;
+import org.jboss.msc.inject.Injector;
+import org.jboss.msc.service.Service;
+import org.jboss.msc.service.StartContext;
+import org.jboss.msc.service.StartException;
+import org.jboss.msc.service.StopContext;
+import org.jboss.msc.value.ImmediateValue;
+import org.jboss.msc.value.InjectedValue;
+import org.teiid.transport.ClientServiceRegistry;
+
+
+class ClientServiceRegistryReferenceFactoryService implements Service<ManagedReferenceFactory>, ManagedReferenceFactory {
+ private final InjectedValue<ClientServiceRegistry> csrInjector = new InjectedValue<ClientServiceRegistry>();
+
+ private ManagedReference reference;
+
+ public synchronized void start(StartContext startContext) throws StartException {
+ reference = new ValueManagedReference(new ImmediateValue<Object>(csrInjector.getValue()));
+ }
+
+ public synchronized void stop(StopContext stopContext) {
+ reference = null;
+ }
+
+ public synchronized ManagedReferenceFactory getValue() throws IllegalStateException, IllegalArgumentException {
+ return this;
+ }
+
+ public synchronized ManagedReference getReference() {
+ return reference;
+ }
+
+ public Injector<ClientServiceRegistry> getCSRInjector() {
+ return csrInjector;
+ }
+}
Property changes on: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/ClientServiceRegistryReferenceFactoryService.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Deleted: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Configuration.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Configuration.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Configuration.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,128 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * Copyright 2011, Red Hat, Inc., and individual contributors
- * as indicated by the @author tags. See the copyright.txt file in the
- * distribution for a full listing of individual contributors.
- *
- * This is free software; you can redistribute it and/or modify it
- * under the terms of the GNU Lesser General Public License as
- * published by the Free Software Foundation; either version 2.1 of
- * the License, or (at your option) any later version.
- *
- * This software is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this software; if not, write to the Free
- * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
- * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
- */
-package org.teiid.jboss;
-
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-
-import org.jboss.dmr.ModelNode;
-import org.jboss.dmr.ModelType;
-
-class Configuration {
- public static final String BUFFER_SERVICE = "buffer-service";//$NON-NLS-1$
- public static final String PREPAREDPLAN_CACHE = "preparedplan-cache";//$NON-NLS-1$
- public static final String RESULTSET_CACHE = "resultset-cache";//$NON-NLS-1$
- public static final String QUERY_ENGINE = "query-engine";//$NON-NLS-1$
- public static final String JDBC = "jdbc";//$NON-NLS-1$
- public static final String ODBC = "odbc"; //$NON-NLS-1$
- public static final String TRANSLATOR = "translator"; //$NON-NLS-1$
-
- // Query-ENGINE
- public static final String ASYNC_THREAD_GROUP = "async-thread-group";//$NON-NLS-1$
- public static final String MAX_THREADS = "max-threads";//$NON-NLS-1$
- public static final String MAX_ACTIVE_PLANS = "max-active-plans";//$NON-NLS-1$
- public static final String USER_REQUEST_SOURCE_CONCURRENCY = "thread-count-for-source-concurrency";//$NON-NLS-1$
- public static final String TIME_SLICE_IN_MILLI = "time-slice-in-millseconds";//$NON-NLS-1$
- public static final String MAX_ROWS_FETCH_SIZE = "max-row-fetch-size";//$NON-NLS-1$
- public static final String LOB_CHUNK_SIZE_IN_KB = "lob-chunk-size-in-kb";//$NON-NLS-1$
- public static final String QUERY_THRESHOLD_IN_SECS = "query-threshold-in-seconds";//$NON-NLS-1$
- public static final String MAX_SOURCE_ROWS = "max-source-rows-allowed";//$NON-NLS-1$
- public static final String EXCEPTION_ON_MAX_SOURCE_ROWS = "exception-on-max-source-rows";//$NON-NLS-1$
- public static final String MAX_ODBC_LOB_SIZE_ALLOWED = "max-odbc-lob-size-allowed";//$NON-NLS-1$
- public static final String OBJECT_REPLICATOR = "object-replicator";//$NON-NLS-1$
- public static final String DETECTING_CHANGE_EVENTS = "detect-change-events";//$NON-NLS-1$
- public static final String SECURITY_DOMAIN = "security-domain";//$NON-NLS-1$
- public static final String MAX_SESSIONS_ALLOWED = "max-sessions-allowed";//$NON-NLS-1$
- public static final String SESSION_EXPIRATION_TIME_LIMIT = "sessions-expiration-timelimit";//$NON-NLS-1$
- public static final String ALLOW_ENV_FUNCTION = "allow-env-function";//$NON-NLS-1$
- public static final String AUTHORIZATION_VALIDATOR_MODULE = "authorization-validator-module"; //$NON-NLS-1$
- public static final String POLICY_DECIDER_MODULE = "policy-decider-module"; //$NON-NLS-1$
-
-
- // Buffer Manager
- public static final String USE_DISK = "use-disk";//$NON-NLS-1$
- public static final String PROCESSOR_BATCH_SIZE = "processor-batch-size";//$NON-NLS-1$
- public static final String CONNECTOR_BATCH_SIZE = "connector-batch-size";//$NON-NLS-1$
- public static final String MAX_PROCESSING_KB = "max-processing-kb";//$NON-NLS-1$
- public static final String MAX_RESERVED_KB = "max-reserve-kb";//$NON-NLS-1$
- public static final String MAX_FILE_SIZE = "max-file-size";//$NON-NLS-1$
- public static final String MAX_BUFFER_SPACE = "max-buffer-space";//$NON-NLS-1$
- public static final String MAX_OPEN_FILES = "max-open-files";//$NON-NLS-1$
-
- //cache-config
- public static final String MAX_ENTRIES = "max-entries";//$NON-NLS-1$
- public static final String MAX_AGE_IN_SECS = "max-age-in-seconds";//$NON-NLS-1$
- public static final String MAX_STALENESS = "max-staleness";//$NON-NLS-1$
- public static final String ENABLE = "enable";//$NON-NLS-1$
-
- // cache-container
- public static final String NAME = "name";//$NON-NLS-1$
- public static final String CONTAINER_NAME = "container-name";//$NON-NLS-1$
-
- //socket config
- public static final String MAX_SOCKET_THREAD_SIZE = "max-socket-threads";//$NON-NLS-1$
- public static final String IN_BUFFER_SIZE = "input-buffer-size";//$NON-NLS-1$
- public static final String OUT_BUFFER_SIZE = "output-buffer-size";//$NON-NLS-1$
- public static final String SOCKET_BINDING = "socket-binding";//$NON-NLS-1$
- public static final String SSL_MODE = "mode";//$NON-NLS-1$
- public static final String KEY_STORE_FILE = "keystore-name";//$NON-NLS-1$
- public static final String KEY_STORE_PASSWD = "keystore-password";//$NON-NLS-1$
- public static final String KEY_STORE_TYPE = "keystore-type";//$NON-NLS-1$
- public static final String SSL_PROTOCOL = "ssl-protocol";//$NON-NLS-1$
- public static final String KEY_MANAGEMENT_ALG = "keymanagement-algorithm";//$NON-NLS-1$
- public static final String TRUST_FILE = "truststore-name";//$NON-NLS-1$
- public static final String TRUST_PASSWD = "truststore-password";//$NON-NLS-1$
- public static final String AUTH_MODE = "authentication-mode";//$NON-NLS-1$
- public static final String SSL = "ssl";//$NON-NLS-1$
-
- public static final String TRANSLATOR_NAME = "name";//$NON-NLS-1$
- public static final String TRANSLATOR_MODULE = "module";//$NON-NLS-1$
-
- public static final String STACK = "stack";//$NON-NLS-1$
- public static final String CLUSTER_NAME = "cluster-name";//$NON-NLS-1$
-
- public static final String ENGINE_NAME = "name";//$NON-NLS-1$
-
- public static final String DESC = ".describe"; //$NON-NLS-1$
- static void addAttribute(ModelNode node, String name, String type, String description, ModelType dataType, boolean required, String defaultValue) {
- node.get(type, name, TYPE).set(dataType);
- node.get(type, name, DESCRIPTION).set(description);
- node.get(type, name, REQUIRED).set(required);
- node.get(type, name, MAX_OCCURS).set(1);
- if (defaultValue != null) {
- if (ModelType.INT.equals(dataType)) {
- node.get(type, name, DEFAULT).set(Integer.parseInt(defaultValue));
- }
- else if (ModelType.BOOLEAN.equals(dataType)) {
- node.get(type, name, DEFAULT).set(Boolean.parseBoolean(defaultValue));
- }
- else if (ModelType.LONG.equals(dataType)) {
- node.get(type, name, DEFAULT).set(Long.parseLong(defaultValue));
- }
- else {
- node.get(type, name, DEFAULT).set(defaultValue);
- }
- }
- //TODO: add "allowed" values
- }
-}
-
-
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Element.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Element.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Element.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -22,117 +22,132 @@
package org.teiid.jboss;
-import static org.teiid.jboss.Configuration.*;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DEFAULT;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.MAX_OCCURS;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REQUIRED;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.TYPE;
import java.util.HashMap;
import java.util.Map;
+import java.util.ResourceBundle;
+import java.util.Set;
+import org.jboss.dmr.ModelNode;
+import org.jboss.dmr.ModelType;
+import org.teiid.core.TeiidRuntimeException;
+
+@SuppressWarnings("nls")
enum Element {
// must be first
UNKNOWN(null),
// VM wide elements
- ASYNC_THREAD_GROUP_ELEMENT(ASYNC_THREAD_GROUP),
- ALLOW_ENV_FUNCTION_ELEMENT(ALLOW_ENV_FUNCTION),
- POLICY_DECIDER_MODULE_ELEMENT(POLICY_DECIDER_MODULE),
- BUFFER_SERVICE_ELEMENT(BUFFER_SERVICE),
- PREPAREDPLAN_CACHE_ELEMENT(PREPAREDPLAN_CACHE),
- RESULTSET_CACHE_ELEMENT(RESULTSET_CACHE),
- OBJECT_REPLICATOR_ELEMENT(OBJECT_REPLICATOR),
- QUERY_ENGINE_ELEMENT(QUERY_ENGINE),
-
- // Query-ENGINE
- ENGINE_NAME_ATTRIBUTE(ENGINE_NAME),
- MAX_THREADS_ELEMENT(MAX_THREADS),
- MAX_ACTIVE_PLANS_ELEMENT(MAX_ACTIVE_PLANS),
- USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT(USER_REQUEST_SOURCE_CONCURRENCY),
- TIME_SLICE_IN_MILLI_ELEMENT(TIME_SLICE_IN_MILLI),
- MAX_ROWS_FETCH_SIZE_ELEMENT(MAX_ROWS_FETCH_SIZE),
- LOB_CHUNK_SIZE_IN_KB_ELEMENT(LOB_CHUNK_SIZE_IN_KB),
- AUTHORIZATION_VALIDATOR_MODULE_ELEMENT(AUTHORIZATION_VALIDATOR_MODULE),
- QUERY_THRESHOLD_IN_SECS_ELEMENT(QUERY_THRESHOLD_IN_SECS),
- MAX_SOURCE_ROWS_ELEMENT(MAX_SOURCE_ROWS),
- EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT(EXCEPTION_ON_MAX_SOURCE_ROWS),
- MAX_ODBC_LOB_SIZE_ALLOWED_ELEMENT(MAX_ODBC_LOB_SIZE_ALLOWED),
- DETECTING_CHANGE_EVENTS_ELEMENT(DETECTING_CHANGE_EVENTS),
- MAX_SESSIONS_ALLOWED_ELEMENT(MAX_SESSIONS_ALLOWED),
- SESSION_EXPIRATION_TIME_LIMIT_ELEMENT(SESSION_EXPIRATION_TIME_LIMIT),
- SECURITY_DOMAIN_ELEMENT(SECURITY_DOMAIN),
- JDBC_ELEMENT(JDBC),
- ODBC_ELEMENT(ODBC),
+ ASYNC_THREAD_POOL_ELEMENT("async-thread-pool", "async-thread-pool", ModelType.STRING, true, null),
+ ALLOW_ENV_FUNCTION_ELEMENT("allow-env-function", "allow-env-function", ModelType.BOOLEAN, false, "false"),
+
+ MAX_THREADS_ELEMENT("max-threads", "max-threads", ModelType.INT, false, "64"),
+ MAX_ACTIVE_PLANS_ELEMENT("max-active-plans", "max-active-plans", ModelType.INT, false, "20"),
+ USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT("thread-count-for-source-concurrency", "thread-count-for-source-concurrency", ModelType.INT, false, "0"),
+ TIME_SLICE_IN_MILLI_ELEMENT("time-slice-in-millseconds", "time-slice-in-millseconds", ModelType.INT, false, "2000"),
+ MAX_ROWS_FETCH_SIZE_ELEMENT("max-row-fetch-size", "max-row-fetch-size", ModelType.INT, false, "20480"),
+ LOB_CHUNK_SIZE_IN_KB_ELEMENT("lob-chunk-size-in-kb", "lob-chunk-size-in-kb", ModelType.INT, false, "100"),
+ QUERY_THRESHOLD_IN_SECS_ELEMENT("query-threshold-in-seconds", "query-threshold-in-seconds", ModelType.INT, false, "600"),
+ MAX_SOURCE_ROWS_ELEMENT("max-source-rows-allowed", "max-source-rows-allowed", ModelType.INT, false, "-1"),
+ EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT("exception-on-max-source-rows", "exception-on-max-source-rows", ModelType.BOOLEAN, false, "true"),
+ DETECTING_CHANGE_EVENTS_ELEMENT("detect-change-events", "detect-change-events", ModelType.BOOLEAN, false, "true"),
+ POLICY_DECIDER_ELEMENT("policy-decider"),
+ POLICY_DECIDER_MODULE_ATTRIBUTE("module", "policy-decider-module", ModelType.STRING, false, null),
+
+ AUTHORIZATION_VALIDATOR_ELEMENT("authorization-validator"),
+ AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE("module", "authorization-validator-module", ModelType.STRING, false, null),
+
// buffer manager
- USE_DISK_ELEMENT(USE_DISK, BUFFER_SERVICE),
- PROCESSOR_BATCH_SIZE_ELEMENT(PROCESSOR_BATCH_SIZE, BUFFER_SERVICE),
- CONNECTOR_BATCH_SIZE_ELEMENT(CONNECTOR_BATCH_SIZE, BUFFER_SERVICE),
- MAX_PROCESSING_KB_ELEMENT(MAX_PROCESSING_KB, BUFFER_SERVICE),
- MAX_RESERVED_KB_ELEMENT(MAX_RESERVED_KB, BUFFER_SERVICE),
- MAX_FILE_SIZE_ELEMENT(MAX_FILE_SIZE, BUFFER_SERVICE),
- MAX_BUFFER_SPACE_ELEMENT(MAX_BUFFER_SPACE, BUFFER_SERVICE),
- MAX_OPEN_FILES_ELEMENT(MAX_OPEN_FILES, BUFFER_SERVICE),
+ BUFFER_SERVICE_ELEMENT("buffer-service"),
+ USE_DISK_ATTRIBUTE("use-disk", "buffer-service-use-disk", ModelType.BOOLEAN, false, "true"),
+ PROCESSOR_BATCH_SIZE_ATTRIBUTE("processor-batch-size", "buffer-service-processor-batch-size", ModelType.INT, false, "512"),
+ CONNECTOR_BATCH_SIZE_ATTRIBUTE("connector-batch-size", "buffer-service-connector-batch-size", ModelType.INT, false, "1024"),
+ MAX_PROCESSING_KB_ATTRIBUTE("max-processing-kb", "buffer-service-max-processing-kb", ModelType.INT, false, "-1"),
+ MAX_RESERVED_KB_ATTRIBUTE("max-reserve-kb", "buffer-service-max-reserve-kb", ModelType.INT, false, "-1"),
+ MAX_FILE_SIZE_ATTRIBUTE("max-file-size", "buffer-service-max-file-size", ModelType.LONG, false, "2048"),
+ MAX_BUFFER_SPACE_ATTRIBUTE("max-buffer-space", "buffer-service-max-buffer-space", ModelType.LONG, false, "51200"),
+ MAX_OPEN_FILES_ATTRIBUTE("max-open-files", "buffer-service-max-open-files", ModelType.INT, false, "64"),
//prepared-plan-cache-config
- PPC_MAX_ENTRIES_ATTRIBUTE(MAX_ENTRIES, PREPAREDPLAN_CACHE),
- PPC_MAX_AGE_IN_SECS_ATTRIBUTE(MAX_AGE_IN_SECS, PREPAREDPLAN_CACHE),
- PPC_MAX_STALENESS_ATTRIBUTE(MAX_STALENESS, PREPAREDPLAN_CACHE),
+ PREPAREDPLAN_CACHE_ELEMENT("preparedplan-cache"),
+ PPC_MAX_ENTRIES_ATTRIBUTE("max-entries", "preparedplan-cache-max-entries", ModelType.INT, false, "512"),
+ PPC_MAX_AGE_IN_SECS_ATTRIBUTE("max-age-in-seconds", "preparedplan-cache-max-age-in-seconds", ModelType.INT, false, "28800"),
+ PPC_MAX_STALENESS_ATTRIBUTE("max-staleness", "preparedplan-cache-max-staleness", ModelType.INT, false, "0"),
-
// Object Replicator
- OR_STACK_ATTRIBUTE(STACK, OBJECT_REPLICATOR),
- OR_CLUSTER_NAME_ATTRIBUTE(CLUSTER_NAME, OBJECT_REPLICATOR),
+ OBJECT_REPLICATOR_ELEMENT("object-replicator"),
+ OR_STACK_ATTRIBUTE("stack", "object-replicator-stack", ModelType.STRING, false, null),
+ OR_CLUSTER_NAME_ATTRIBUTE("cluster-name", "object-replicator-cluster-name", ModelType.STRING, false, null),
- // Result set cache
- RSC_ENABLE_ATTRIBUTE(ENABLE, RESULTSET_CACHE),
- RSC_NAME_ELEMENT(NAME, RESULTSET_CACHE),
- RSC_CONTAINER_NAME_ELEMENT(CONTAINER_NAME, RESULTSET_CACHE),
- RSC_MAX_STALENESS_ELEMENT(MAX_STALENESS, RESULTSET_CACHE),
+ // Result set cache
+ RESULTSET_CACHE_ELEMENT("resultset-cache"),
+ RSC_ENABLE_ATTRIBUTE("enable", "resultset-cache-enable", ModelType.BOOLEAN, false, null),
+ RSC_NAME_ELEMENT("name", "resultset-cache-name", ModelType.STRING, false, "resultset"),
+ RSC_CONTAINER_NAME_ELEMENT("container-name", "resultset-cache-container-name", ModelType.STRING, false, null),
+ RSC_MAX_STALENESS_ELEMENT("max-staleness", "resultset-cache-max-staleness", ModelType.INT, false, "60"),
- //socket config
- JDBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE(MAX_SOCKET_THREAD_SIZE,JDBC),
- JDBC_IN_BUFFER_SIZE_ATTRIBUTE(IN_BUFFER_SIZE,JDBC),
- JDBC_OUT_BUFFER_SIZE_ATTRIBUTE(OUT_BUFFER_SIZE,JDBC),
- JDBC_SOCKET_BINDING_ATTRIBUTE(SOCKET_BINDING,JDBC),
+ //transport
+ TRANSPORT_ELEMENT("transport"),
+ TRANSPORT_PROTOCOL_ATTRIBUTE("protocol", "protocol", ModelType.STRING, false, "teiid"),
+ TRANSPORT_NAME_ATTRIBUTE("name", "name", ModelType.STRING, true, null),
+ TRANSPORT_SOCKET_BINDING_ATTRIBUTE("socket-binding", "socket-binding", ModelType.STRING, true, null),
+ TRANSPORT_MAX_SOCKET_THREADS_ATTRIBUTE("max-socket-threads", "max-socket-threads", ModelType.INT, false, "0"),
+ TRANSPORT_IN_BUFFER_SIZE_ATTRIBUTE("input-buffer-size", "input-buffer-size",ModelType.INT, false, "0"),
+ TRANSPORT_OUT_BUFFER_SIZE_ATTRIBUTE("output-buffer-size", "output-buffer-size", ModelType.INT, false, "0"),
- JDBC_SSL_ELEMENT(SSL, JDBC),
- JDBC_SSL_MODE_ELEMENT(SSL_MODE,JDBC,SSL),
- JDBC_KEY_STORE_FILE_ELEMENT(KEY_STORE_FILE,JDBC,SSL),
- JDBC_KEY_STORE_PASSWD_ELEMENT(KEY_STORE_PASSWD,JDBC,SSL),
- JDBC_KEY_STORE_TYPE_ELEMENT(KEY_STORE_TYPE,JDBC,SSL),
- JDBC_SSL_PROTOCOL_ELEMENT(SSL_PROTOCOL,JDBC,SSL),
- JDBC_KEY_MANAGEMENT_ALG_ELEMENT(KEY_MANAGEMENT_ALG,JDBC,SSL),
- JDBC_TRUST_FILE_ELEMENT(TRUST_FILE,JDBC,SSL),
- JDBC_TRUST_PASSWD_ELEMENT(TRUST_PASSWD,JDBC,SSL),
- JDBC_AUTH_MODE_ELEMENT(AUTH_MODE,JDBC,SSL),
-
+ AUTHENTICATION_ELEMENT("authentication"),
+ AUTHENTICATION_SECURITY_DOMAIN_ATTRIBUTE("security-domain", "authentication-security-domain", ModelType.STRING, false, null),
+ AUTHENTICATION_MAX_SESSIONS_ALLOWED_ATTRIBUTE("max-sessions-allowed", "authentication-max-sessions-allowed",ModelType.INT, false, "5000"),
+ AUTHENTICATION_SESSION_EXPIRATION_TIME_LIMIT_ATTRIBUTE("sessions-expiration-timelimit", "authentication-sessions-expiration-timelimit", ModelType.INT, false, "0"),
- ODBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE(MAX_SOCKET_THREAD_SIZE,ODBC),
- ODBC_IN_BUFFER_SIZE_ATTRIBUTE(IN_BUFFER_SIZE,ODBC),
- ODBC_OUT_BUFFER_SIZE_ATTRIBUTE(OUT_BUFFER_SIZE,ODBC),
- ODBC_SOCKET_BINDING_ATTRIBUTE(SOCKET_BINDING,ODBC),
+ PG_ELEMENT("pg"), //$NON-NLS-1$
+ PG_MAX_LOB_SIZE_ALLOWED_ELEMENT("max-lob-size-in-bytes", "pg-max-lob-size-in-bytes", ModelType.INT, false, "5242880"), //$NON-NLS-1$ //$NON-NLS-2$
+ PG_AUTHENTICATION_TYPE_ATTRIBUTE("type", "pg-auth-type", ModelType.STRING, false, "CLEARTEXT"),
- ODBC_SSL_ELEMENT(SSL, ODBC),
- ODBC_SSL_MODE_ELEMENT(SSL_MODE,ODBC,SSL),
- ODBC_KEY_STORE_FILE_ELEMENT(KEY_STORE_FILE,ODBC,SSL),
- ODBC_KEY_STORE_PASSWD_ELEMENT(KEY_STORE_PASSWD,ODBC,SSL),
- ODBC_KEY_STORE_TYPE_ELEMENT(KEY_STORE_TYPE,ODBC,SSL),
- ODBC_SSL_PROTOCOL_ELEMENT(SSL_PROTOCOL,ODBC,SSL),
- ODBC_KEY_MANAGEMENT_ALG_ELEMENT(KEY_MANAGEMENT_ALG,ODBC,SSL),
- ODBC_TRUST_FILE_ELEMENT(TRUST_FILE,ODBC,SSL),
- ODBC_TRUST_PASSWD_ELEMENT(TRUST_PASSWD,ODBC,SSL),
- ODBC_AUTH_MODE_ELEMENT(AUTH_MODE,ODBC,SSL),
-
+ SSL_ELEMENT("ssl"),
+ SSL_MODE_ATTRIBUTE("mode", "ssl-mode", ModelType.STRING, false, "login"),
+ SSL_AUTH_MODE_ATTRIBUTE("authentication-mode", "ssl-authentication-mode", ModelType.STRING, false, "anonymous"),
+ SSL_SSL_PROTOCOL_ATTRIBUTE("ssl-protocol", "ssl-ssl-protocol", ModelType.STRING, false, "SSLv3"),
+ SSL_KEY_MANAGEMENT_ALG_ATTRIBUTE("keymanagement-algorithm", "ssl-keymanagement-algorithm", ModelType.STRING, false, null),
+ SSL_KETSTORE_ELEMENT("keystore"),
+ SSL_KETSTORE_NAME_ATTRIBUTE("name", "keystore-name", ModelType.STRING, false, null),
+ SSL_KETSTORE_PASSWORD_ATTRIBUTE("password", "keystore-password", ModelType.STRING, false, null),
+ SSL_KETSTORE_TYPE_ATTRIBUTE("type", "keystore-type", ModelType.STRING, false, "JKS"),
+ SSL_TRUSTSTORE_ELEMENT("truststore"),
+ SSL_TRUSTSTORE_NAME_ATTRIBUTE("name", "truststore-name", ModelType.STRING, false, null),
+ SSL_TRUSTSTORE_PASSWORD_ATTRIBUTE("password", "truststore-password", ModelType.STRING, false, null),
// Translator
- TRANSLATOR_ELEMENT(TRANSLATOR),
- TRANSLATOR_NAME_ATTRIBUTE(TRANSLATOR_NAME),
- TRANSLATOR_MODULE_ATTRIBUTE(TRANSLATOR_MODULE);
+ TRANSLATOR_ELEMENT("translator"),
+ TRANSLATOR_NAME_ATTRIBUTE("name", "name", ModelType.STRING, true, null),
+ TRANSLATOR_MODULE_ATTRIBUTE("module", "module", ModelType.STRING, true, null);
private final String name;
- private String[] prefix;
+ private final String modelName;
+ private final boolean required;
+ private final ModelType modelType;
+ private final String defaultValue;
- Element(final String name, String... prefix) {
+ private Element(String name) {
+ this.name = name;
+ this.modelName = name;
+ this.required = false;
+ this.modelType = null;
+ this.defaultValue = null;
+ }
+
+ Element(final String name, String modelName, ModelType type, boolean required, String defltValue) {
this.name = name;
- this.prefix = prefix;
+ this.modelName = modelName;
+ this.modelType = type;
+ this.required = required;
+ this.defaultValue = defltValue;
}
/**
@@ -145,29 +160,9 @@
}
public String getModelName() {
- return buildModelName(this.name, this.prefix);
+ return this.modelName;
}
- private static String buildModelName(String name, String... prefix) {
- if (prefix == null || prefix.length == 0) {
- return name;
- }
-
- StringBuilder sb = new StringBuilder();
- sb.append(prefix[0]);
- for (int i = 1; i < prefix.length; i++) {
- sb.append("-"); //$NON-NLS-1$
- sb.append(prefix[i]);
- }
- sb.append("-"); //$NON-NLS-1$
- sb.append(name);
- return sb.toString();
- }
-
- public String[] getPrefix() {
- return this.prefix;
- }
-
private static final Map<String, Element> elements;
static {
@@ -179,10 +174,106 @@
elements = map;
}
- public static Element forName(String localName, String... prefix) {
- String modelName = buildModelName(localName, prefix);
+ public static Element forName(String localName, Element parentNode) {
+ String modelName = parentNode.getLocalName()+"-"+localName;
final Element element = elements.get(modelName);
return element == null ? UNKNOWN : element;
}
+
+ public static Element forName(String localName) {
+ final Element element = elements.get(localName);
+ return element == null ? UNKNOWN : element;
+ }
+
+ public void describe(ModelNode node, String type, ResourceBundle bundle) {
+ String name = getModelName();
+ node.get(type, name, TYPE).set(this.modelType);
+ node.get(type, name, DESCRIPTION).set(getDescription(bundle));
+ node.get(type, name, REQUIRED).set(this.required);
+ node.get(type, name, MAX_OCCURS).set(1);
+
+ if (this.defaultValue != null) {
+ if (ModelType.INT == this.modelType) {
+ node.get(type, name, DEFAULT).set(Integer.parseInt(this.defaultValue));
+ }
+ else if (ModelType.BOOLEAN == this.modelType) {
+ node.get(type, name, DEFAULT).set(Boolean.parseBoolean(this.defaultValue));
+ }
+ else if (ModelType.LONG == this.modelType) {
+ node.get(type, name, DEFAULT).set(Long.parseLong(this.defaultValue));
+ }
+ else if (ModelType.STRING == this.modelType) {
+ node.get(type, name, DEFAULT).set(this.defaultValue);
+ }
+ else {
+ throw new TeiidRuntimeException();
+ }
+ }
+ }
+
+ public void populate(ModelNode operation, ModelNode model) {
+ if (getModelName() == null) {
+ return;
+ }
+
+ if (operation.hasDefined(getModelName())) {
+ if (ModelType.STRING == this.modelType) {
+ model.get(getModelName()).set(operation.get(getModelName()).asString());
+ }
+ else if (ModelType.INT == this.modelType) {
+ model.get(getModelName()).set(operation.get(getModelName()).asInt());
+ }
+ else if (ModelType.LONG == this.modelType) {
+ model.get(getModelName()).set(operation.get(getModelName()).asLong());
+ }
+ else if (ModelType.BOOLEAN == this.modelType) {
+ model.get(getModelName()).set(operation.get(getModelName()).asBoolean());
+ }
+ else {
+ throw new TeiidRuntimeException();
+ }
+ }
+ }
+
+ public boolean isDefined(ModelNode node) {
+ return node.hasDefined(getModelName());
+ }
+
+ public int asInt(ModelNode node) {
+ return node.get(getModelName()).asInt();
+ }
+
+ public long asLong(ModelNode node) {
+ return node.get(getModelName()).asLong();
+ }
+
+ public String asString(ModelNode node) {
+ return node.get(getModelName()).asString();
+ }
+
+ public boolean asBoolean(ModelNode node) {
+ return node.get(getModelName()).asBoolean();
+ }
+
+ public boolean isLike(ModelNode node) {
+ Set<String> keys = node.keys();
+ for(String key:keys) {
+ if (key.startsWith(this.name)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public String getDescription(ResourceBundle bundle) {
+ return bundle.getString(this.modelName+".describe");
+ }
+
+ public boolean sameAsDefault(String value) {
+ if (this.defaultValue == null) {
+ return (value == null);
+ }
+ return this.defaultValue.equalsIgnoreCase(value);
+ }
}
Deleted: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineAdd.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineAdd.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineAdd.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,424 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership. Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- *
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- *
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-package org.teiid.jboss;
-
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-import static org.teiid.jboss.Configuration.DESC;
-import static org.teiid.jboss.Configuration.addAttribute;
-
-import java.util.List;
-import java.util.Locale;
-import java.util.ResourceBundle;
-
-import javax.resource.spi.XATerminator;
-import javax.resource.spi.work.WorkManager;
-import javax.transaction.TransactionManager;
-
-import org.jboss.as.controller.*;
-import org.jboss.as.controller.descriptions.DescriptionProvider;
-import org.jboss.as.naming.ManagedReferenceFactory;
-import org.jboss.as.naming.NamingStore;
-import org.jboss.as.naming.deployment.ContextNames;
-import org.jboss.as.naming.service.BinderService;
-import org.jboss.as.network.SocketBinding;
-import org.jboss.as.security.plugins.SecurityDomainContext;
-import org.jboss.dmr.ModelNode;
-import org.jboss.dmr.ModelType;
-import org.jboss.msc.inject.ConcurrentMapInjector;
-import org.jboss.msc.service.*;
-import org.jboss.msc.service.ServiceBuilder.DependencyType;
-import org.jboss.msc.value.InjectedValue;
-import org.teiid.deployers.SystemVDBDeployer;
-import org.teiid.deployers.VDBRepository;
-import org.teiid.dqp.internal.datamgr.TranslatorRepository;
-import org.teiid.dqp.internal.process.AuthorizationValidator;
-import org.teiid.dqp.internal.process.SessionAwareCache;
-import org.teiid.jboss.deployers.RuntimeEngineDeployer;
-import org.teiid.logging.LogConstants;
-import org.teiid.logging.LogManager;
-import org.teiid.query.ObjectReplicator;
-import org.teiid.services.BufferServiceImpl;
-import org.teiid.transport.ClientServiceRegistry;
-import org.teiid.transport.LocalServerConnection;
-import org.teiid.transport.SSLConfiguration;
-import org.teiid.transport.SocketConfiguration;
-
-class QueryEngineAdd extends AbstractAddStepHandler implements DescriptionProvider {
-
- @Override
- public ModelNode getModelDescription(Locale locale) {
- final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
-
- final ModelNode node = new ModelNode();
- node.get(OPERATION_NAME).set(ADD);
- node.get(DESCRIPTION).set("engine.add"); //$NON-NLS-1$
-
- describeQueryEngine(node, REQUEST_PROPERTIES, bundle);
- return node;
- }
-
- @Override
- protected void populateModel(ModelNode operation, ModelNode model) {
- populate(operation, model);
- }
-
- @Override
- protected void performRuntime(final OperationContext context, final ModelNode operation, final ModelNode model,
- final ServiceVerificationHandler verificationHandler, final List<ServiceController<?>> newControllers) throws OperationFailedException {
-
- ServiceTarget target = context.getServiceTarget();
-
- final JBossLifeCycleListener shutdownListener = new JBossLifeCycleListener();
-
- final ModelNode address = operation.require(OP_ADDR);
- final PathAddress pathAddress = PathAddress.pathAddress(address);
- final String engineName = pathAddress.getLastElement().getValue();
-
- // now build the engine
- final RuntimeEngineDeployer engine = buildQueryEngine(engineName, operation);
- engine.setSecurityHelper(new JBossSecurityHelper());
-
- SocketConfiguration jdbc = buildSocketConfiguration(Configuration.JDBC, operation);
- if (jdbc != null) {
- engine.setJdbcSocketConfiguration(jdbc);
- }
-
- SocketConfiguration odbc = buildSocketConfiguration(Configuration.ODBC, operation);
- if (odbc != null) {
- engine.setOdbcSocketConfiguration(odbc);
- }
-
- ServiceBuilder<ClientServiceRegistry> engineBuilder = target.addService(TeiidServiceNames.engineServiceName(engine.getName()), engine);
- engineBuilder.addDependency(ServiceName.JBOSS.append("connector", "workmanager"), WorkManager.class, engine.getWorkManagerInjector()); //$NON-NLS-1$ //$NON-NLS-2$
- engineBuilder.addDependency(ServiceName.JBOSS.append("txn", "XATerminator"), XATerminator.class, engine.getXaTerminatorInjector()); //$NON-NLS-1$ //$NON-NLS-2$
- engineBuilder.addDependency(ServiceName.JBOSS.append("txn", "TransactionManager"), TransactionManager.class, engine.getTxnManagerInjector()); //$NON-NLS-1$ //$NON-NLS-2$
- engineBuilder.addDependency(TeiidServiceNames.BUFFER_MGR, BufferServiceImpl.class, engine.getBufferServiceInjector());
- engineBuilder.addDependency(TeiidServiceNames.SYSTEM_VDB, SystemVDBDeployer.class, new InjectedValue<SystemVDBDeployer>());
- engineBuilder.addDependency(TeiidServiceNames.TRANSLATOR_REPO, TranslatorRepository.class, engine.getTranslatorRepositoryInjector());
- engineBuilder.addDependency(TeiidServiceNames.VDB_REPO, VDBRepository.class, engine.getVdbRepositoryInjector());
- engineBuilder.addDependency(TeiidServiceNames.AUTHORIZATION_VALIDATOR, AuthorizationValidator.class, engine.getAuthorizationValidatorInjector());
- engineBuilder.addDependency(TeiidServiceNames.CACHE_RESULTSET, SessionAwareCache.class, engine.getResultSetCacheInjector());
- engineBuilder.addDependency(TeiidServiceNames.CACHE_PREPAREDPLAN, SessionAwareCache.class, engine.getPreparedPlanCacheInjector());
- engineBuilder.addDependency(DependencyType.OPTIONAL, TeiidServiceNames.OBJECT_REPLICATOR, ObjectReplicator.class, engine.getObjectReplicatorInjector());
-
- if (jdbc != null) {
- engineBuilder.addDependency(ServiceName.JBOSS.append("binding", jdbc.getSocketBinding()), SocketBinding.class, engine.getJdbcSocketBindingInjector()); //$NON-NLS-1$
- }
-
- if (odbc != null) {
- engineBuilder.addDependency(ServiceName.JBOSS.append("binding", odbc.getSocketBinding()), SocketBinding.class, engine.getOdbcSocketBindingInjector()); //$NON-NLS-1$
- }
-
- // register a JNDI name, this looks hard.
- final QueryEngineReferenceFactoryService referenceFactoryService = new QueryEngineReferenceFactoryService();
- final ServiceName referenceFactoryServiceName = TeiidServiceNames.engineServiceName(engine.getName()).append("reference-factory"); //$NON-NLS-1$
- final ServiceBuilder<?> referenceBuilder = target.addService(referenceFactoryServiceName,referenceFactoryService);
- referenceBuilder.addDependency(TeiidServiceNames.engineServiceName(engine.getName()), RuntimeEngineDeployer.class, referenceFactoryService.getQueryEngineInjector());
- referenceBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
-
- final ContextNames.BindInfo bindInfo = ContextNames.bindInfoFor(LocalServerConnection.TEIID_RUNTIME_CONTEXT+engine.getName());
- final BinderService engineBinderService = new BinderService(bindInfo.getBindName());
- final ServiceBuilder<?> engineBinderBuilder = target.addService(bindInfo.getBinderServiceName(), engineBinderService);
- engineBinderBuilder.addDependency(referenceFactoryServiceName, ManagedReferenceFactory.class, engineBinderService.getManagedObjectInjector());
- engineBinderBuilder.addDependency(bindInfo.getParentContextServiceName(), NamingStore.class, engineBinderService.getNamingStoreInjector());
- engineBinderBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
-
- // add security domains
- if ( operation.hasDefined(Configuration.SECURITY_DOMAIN)) {
- List<ModelNode> domains = operation.get(Configuration.SECURITY_DOMAIN).asList();
- for (ModelNode domain:domains) {
- LogManager.logInfo(LogConstants.CTX_SECURITY, "Security Enabled: true"); //$NON-NLS-1$
- engineBuilder.addDependency(ServiceName.JBOSS.append("security", "security-domain", domain.asString()), SecurityDomainContext.class, new ConcurrentMapInjector<String,SecurityDomainContext>(engine.securityDomains, domain.asString())); //$NON-NLS-1$ //$NON-NLS-2$
- }
- }
-
- engineBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
- ServiceController<ClientServiceRegistry> controller = engineBuilder.install();
- newControllers.add(controller);
- ServiceContainer container = controller.getServiceContainer();
- container.addTerminateListener(shutdownListener);
-
- newControllers.add(referenceBuilder.install());
- newControllers.add(engineBinderBuilder.install());
- }
-
-
- private RuntimeEngineDeployer buildQueryEngine(String engineName, ModelNode node) {
- RuntimeEngineDeployer engine = new RuntimeEngineDeployer(engineName);
-
- if (node.hasDefined(Configuration.MAX_THREADS)) {
- engine.setMaxThreads(node.get(Configuration.MAX_THREADS).asInt());
- }
- if (node.hasDefined(Configuration.MAX_ACTIVE_PLANS)) {
- engine.setMaxActivePlans(node.get(Configuration.MAX_ACTIVE_PLANS).asInt());
- }
- if (node.hasDefined(Configuration.USER_REQUEST_SOURCE_CONCURRENCY)) {
- engine.setUserRequestSourceConcurrency(node.get(Configuration.USER_REQUEST_SOURCE_CONCURRENCY).asInt());
- }
- if (node.hasDefined(Configuration.TIME_SLICE_IN_MILLI)) {
- engine.setTimeSliceInMilli(node.get(Configuration.TIME_SLICE_IN_MILLI).asInt());
- }
- if (node.hasDefined(Configuration.MAX_ROWS_FETCH_SIZE)) {
- engine.setMaxRowsFetchSize(node.get(Configuration.MAX_ROWS_FETCH_SIZE).asInt());
- }
- if (node.hasDefined(Configuration.LOB_CHUNK_SIZE_IN_KB)) {
- engine.setLobChunkSizeInKB(node.get(Configuration.LOB_CHUNK_SIZE_IN_KB).asInt());
- }
- if (node.hasDefined(Configuration.QUERY_THRESHOLD_IN_SECS)) {
- engine.setQueryThresholdInSecs(node.get(Configuration.QUERY_THRESHOLD_IN_SECS).asInt());
- }
- if (node.hasDefined(Configuration.MAX_SOURCE_ROWS)) {
- engine.setMaxSourceRows(node.get(Configuration.MAX_SOURCE_ROWS).asInt());
- }
- if (node.hasDefined(Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS)) {
- engine.setExceptionOnMaxSourceRows(node.get(Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS).asBoolean());
- }
- if (node.hasDefined(Configuration.MAX_ODBC_LOB_SIZE_ALLOWED)) {
- engine.setMaxODBCLobSizeAllowed(node.get(Configuration.MAX_ODBC_LOB_SIZE_ALLOWED).asInt());
- }
- if (node.hasDefined(Configuration.DETECTING_CHANGE_EVENTS)) {
- engine.setDetectingChangeEvents(node.get(Configuration.DETECTING_CHANGE_EVENTS).asBoolean());
- }
- if (node.hasDefined(Configuration.SESSION_EXPIRATION_TIME_LIMIT)) {
- engine.setSessionExpirationTimeLimit(node.get(Configuration.SESSION_EXPIRATION_TIME_LIMIT).asInt());
- }
- if (node.hasDefined(Configuration.MAX_SESSIONS_ALLOWED)) {
- engine.setSessionMaxLimit(node.get(Configuration.MAX_SESSIONS_ALLOWED).asInt());
- }
- if (node.hasDefined(Configuration.SECURITY_DOMAIN)) {
- List<ModelNode> securityDomains = node.get(Configuration.SECURITY_DOMAIN).asList();
- for (ModelNode domain:securityDomains) {
- engine.addSecurityDomain(domain.asString());
- }
- }
- return engine;
- }
-
-
- private SocketConfiguration buildSocketConfiguration(String prefix, ModelNode node) {
-
- if (!node.hasDefined(prefix+Configuration.SOCKET_BINDING)) {
- return null;
- }
-
- SocketConfiguration socket = new SocketConfiguration();
- socket.setSocketBinding(node.require(prefix+Configuration.SOCKET_BINDING).asString());
-
- if (node.hasDefined(prefix+Configuration.MAX_SOCKET_THREAD_SIZE)) {
- socket.setMaxSocketThreads(node.get(prefix+Configuration.MAX_SOCKET_THREAD_SIZE).asInt());
- }
- if (node.hasDefined(prefix+Configuration.IN_BUFFER_SIZE)) {
- socket.setInputBufferSize(node.get(prefix+Configuration.IN_BUFFER_SIZE).asInt());
- }
- if (node.hasDefined(prefix+Configuration.OUT_BUFFER_SIZE)) {
- socket.setOutputBufferSize(node.get(prefix+Configuration.OUT_BUFFER_SIZE).asInt());
- }
-
- SSLConfiguration ssl = new SSLConfiguration();
- ssl.setAuthenticationMode(SSLConfiguration.ANONYMOUS);
-
- String sslPrefix = prefix+ Configuration.SSL +TeiidBootServicesAdd.DASH;
-
- if (node.hasDefined(sslPrefix+Configuration.SSL_MODE)) {
- ssl.setMode(node.get(sslPrefix+Configuration.SSL_MODE).asString());
- }
-
- if (node.hasDefined(sslPrefix+Configuration.KEY_STORE_FILE)) {
- ssl.setKeystoreFilename(node.get(sslPrefix+Configuration.KEY_STORE_FILE).asString());
- }
-
- if (node.hasDefined(sslPrefix+Configuration.KEY_STORE_PASSWD)) {
- ssl.setKeystorePassword(node.get(sslPrefix+Configuration.KEY_STORE_PASSWD).asString());
- }
-
- if (node.hasDefined(sslPrefix+Configuration.KEY_STORE_TYPE)) {
- ssl.setKeystoreType(node.get(sslPrefix+Configuration.KEY_STORE_TYPE).asString());
- }
-
- if (node.hasDefined(sslPrefix+Configuration.SSL_PROTOCOL)) {
- ssl.setSslProtocol(node.get(sslPrefix+Configuration.SSL_PROTOCOL).asString());
- }
- if (node.hasDefined(sslPrefix+Configuration.KEY_MANAGEMENT_ALG)) {
- ssl.setKeymanagementAlgorithm(node.get(sslPrefix+Configuration.KEY_MANAGEMENT_ALG).asString());
- }
- if (node.hasDefined(sslPrefix+Configuration.TRUST_FILE)) {
- ssl.setTruststoreFilename(node.get(sslPrefix+Configuration.TRUST_FILE).asString());
- }
- if (node.hasDefined(sslPrefix+Configuration.TRUST_PASSWD)) {
- ssl.setTruststorePassword(node.get(sslPrefix+Configuration.TRUST_PASSWD).asString());
- }
- if (node.hasDefined(sslPrefix+Configuration.AUTH_MODE)) {
- ssl.setAuthenticationMode(node.get(sslPrefix+Configuration.AUTH_MODE).asString());
- }
- socket.setSSLConfiguration(ssl);
-
- return socket;
- }
-
- static void describeQueryEngine(ModelNode node, String type, ResourceBundle bundle) {
- addAttribute(node, Configuration.MAX_THREADS, type, bundle.getString(Configuration.MAX_THREADS+DESC), ModelType.INT, false, "64"); //$NON-NLS-1$
- addAttribute(node, Configuration.MAX_ACTIVE_PLANS, type, bundle.getString(Configuration.MAX_ACTIVE_PLANS+DESC), ModelType.INT, false, "20"); //$NON-NLS-1$
- addAttribute(node, Configuration.USER_REQUEST_SOURCE_CONCURRENCY, type, bundle.getString(Configuration.USER_REQUEST_SOURCE_CONCURRENCY+DESC), ModelType.INT, false, "0"); //$NON-NLS-1$
- addAttribute(node, Configuration.TIME_SLICE_IN_MILLI, type, bundle.getString(Configuration.TIME_SLICE_IN_MILLI+DESC), ModelType.INT, false, "2000"); //$NON-NLS-1$
- addAttribute(node, Configuration.MAX_ROWS_FETCH_SIZE, type, bundle.getString(Configuration.MAX_ROWS_FETCH_SIZE+DESC), ModelType.INT, false, "20480"); //$NON-NLS-1$
- addAttribute(node, Configuration.LOB_CHUNK_SIZE_IN_KB, type, bundle.getString(Configuration.LOB_CHUNK_SIZE_IN_KB+DESC), ModelType.INT, false, "100"); //$NON-NLS-1$
- addAttribute(node, Configuration.QUERY_THRESHOLD_IN_SECS, type, bundle.getString(Configuration.QUERY_THRESHOLD_IN_SECS+DESC), ModelType.INT, false, "600"); //$NON-NLS-1$
- addAttribute(node, Configuration.MAX_SOURCE_ROWS, type, bundle.getString(Configuration.MAX_SOURCE_ROWS+DESC), ModelType.INT, false, "-1"); //$NON-NLS-1$
- addAttribute(node, Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS, type, bundle.getString(Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS+DESC), ModelType.BOOLEAN, false, "true"); //$NON-NLS-1$
- addAttribute(node, Configuration.MAX_ODBC_LOB_SIZE_ALLOWED, type, bundle.getString(Configuration.MAX_ODBC_LOB_SIZE_ALLOWED+DESC), ModelType.INT, false, "5242880"); //$NON-NLS-1$
- addAttribute(node, Configuration.DETECTING_CHANGE_EVENTS, type, bundle.getString(Configuration.DETECTING_CHANGE_EVENTS+DESC), ModelType.BOOLEAN, false, "true"); //$NON-NLS-1$
-
- //session stuff
- addAttribute(node, Configuration.SECURITY_DOMAIN, type, bundle.getString(Configuration.SECURITY_DOMAIN+DESC), ModelType.LIST, false, null);
- addAttribute(node, Configuration.MAX_SESSIONS_ALLOWED, type, bundle.getString(Configuration.MAX_SESSIONS_ALLOWED+DESC), ModelType.INT, false, "5000"); //$NON-NLS-1$
- addAttribute(node, Configuration.SESSION_EXPIRATION_TIME_LIMIT, type, bundle.getString(Configuration.SESSION_EXPIRATION_TIME_LIMIT+DESC), ModelType.INT, false, "0"); //$NON-NLS-1$
-
- //jdbc
- describeSocketConfig(Configuration.JDBC+TeiidBootServicesAdd.DASH, node, type, bundle);
-
- //odbc
- describeSocketConfig(Configuration.ODBC+TeiidBootServicesAdd.DASH, node, type, bundle);
- }
-
-
- private static void describeSocketConfig(String prefix, ModelNode node, String type, ResourceBundle bundle) {
- addAttribute(node, prefix+Configuration.MAX_SOCKET_THREAD_SIZE, type, bundle.getString(Configuration.MAX_SOCKET_THREAD_SIZE+DESC), ModelType.INT, false, "0"); //$NON-NLS-1$
- addAttribute(node, prefix+Configuration.IN_BUFFER_SIZE, type, bundle.getString(Configuration.IN_BUFFER_SIZE+DESC), ModelType.INT, false, "0"); //$NON-NLS-1$
- addAttribute(node, prefix+Configuration.OUT_BUFFER_SIZE, type, bundle.getString(Configuration.OUT_BUFFER_SIZE+DESC), ModelType.INT, false, "0"); //$NON-NLS-1$
- addAttribute(node, prefix+Configuration.SOCKET_BINDING, type, bundle.getString(Configuration.SOCKET_BINDING+DESC), ModelType.STRING, false, null);
-
- String sslPrefix = prefix+Configuration.SSL+TeiidBootServicesAdd.DASH;
-
- addAttribute(node, sslPrefix+Configuration.SSL_MODE, type, bundle.getString(Configuration.SSL_MODE+DESC), ModelType.STRING, false, "login"); //$NON-NLS-1$
- addAttribute(node, sslPrefix+Configuration.KEY_STORE_FILE, type, bundle.getString(Configuration.KEY_STORE_FILE+DESC), ModelType.STRING, false, null);
- addAttribute(node, sslPrefix+Configuration.KEY_STORE_PASSWD, type, bundle.getString(Configuration.KEY_STORE_PASSWD+DESC), ModelType.STRING, false, null);
- addAttribute(node, sslPrefix+Configuration.KEY_STORE_TYPE, type, bundle.getString(Configuration.KEY_STORE_TYPE+DESC), ModelType.STRING, false, "JKS"); //$NON-NLS-1$
- addAttribute(node, sslPrefix+Configuration.SSL_PROTOCOL, type, bundle.getString(Configuration.SSL_PROTOCOL+DESC), ModelType.STRING, false, "SSLv3"); //$NON-NLS-1$
- addAttribute(node, sslPrefix+Configuration.KEY_MANAGEMENT_ALG, type, bundle.getString(Configuration.KEY_MANAGEMENT_ALG+DESC), ModelType.STRING, false, null);
- addAttribute(node, sslPrefix+Configuration.TRUST_FILE, type, bundle.getString(Configuration.TRUST_FILE+DESC), ModelType.STRING, false, null);
- addAttribute(node, sslPrefix+Configuration.TRUST_PASSWD, type, bundle.getString(Configuration.TRUST_PASSWD+DESC), ModelType.STRING, false, null);
- addAttribute(node, sslPrefix+Configuration.AUTH_MODE, type, bundle.getString(Configuration.AUTH_MODE+DESC), ModelType.STRING, false, "anonymous"); //$NON-NLS-1$
- }
-
- static void populate(ModelNode operation, ModelNode model) {
- //model.get(Configuration.ENGINE_NAME).set(engineName);
-
- if (operation.hasDefined(Configuration.MAX_THREADS)) {
- model.get(Configuration.MAX_THREADS).set(operation.get(Configuration.MAX_THREADS).asInt());
- }
- if (operation.hasDefined(Configuration.MAX_ACTIVE_PLANS)) {
- model.get(Configuration.MAX_ACTIVE_PLANS).set(operation.get(Configuration.MAX_ACTIVE_PLANS).asInt());
- }
- if (operation.hasDefined(Configuration.USER_REQUEST_SOURCE_CONCURRENCY)) {
- model.get(Configuration.USER_REQUEST_SOURCE_CONCURRENCY).set(operation.get(Configuration.USER_REQUEST_SOURCE_CONCURRENCY).asInt());
- }
- if (operation.hasDefined(Configuration.TIME_SLICE_IN_MILLI)) {
- model.get(Configuration.TIME_SLICE_IN_MILLI).set(operation.get(Configuration.TIME_SLICE_IN_MILLI).asInt());
- }
- if (operation.hasDefined(Configuration.MAX_ROWS_FETCH_SIZE)) {
- model.get(Configuration.MAX_ROWS_FETCH_SIZE).set(operation.get(Configuration.MAX_ROWS_FETCH_SIZE).asInt());
- }
- if (operation.hasDefined(Configuration.LOB_CHUNK_SIZE_IN_KB)) {
- model.get(Configuration.LOB_CHUNK_SIZE_IN_KB).set(operation.get(Configuration.LOB_CHUNK_SIZE_IN_KB).asInt());
- }
- if (operation.hasDefined(Configuration.QUERY_THRESHOLD_IN_SECS)) {
- model.get(Configuration.QUERY_THRESHOLD_IN_SECS).set(operation.get(Configuration.QUERY_THRESHOLD_IN_SECS).asInt());
- }
- if (operation.hasDefined(Configuration.MAX_SOURCE_ROWS)) {
- model.get(Configuration.MAX_SOURCE_ROWS).set(operation.get(Configuration.MAX_SOURCE_ROWS).asInt());
- }
- if (operation.hasDefined(Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS)) {
- model.get(Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS).set(operation.get(Configuration.EXCEPTION_ON_MAX_SOURCE_ROWS).asBoolean());
- }
- if (operation.hasDefined(Configuration.MAX_ODBC_LOB_SIZE_ALLOWED)) {
- model.get(Configuration.MAX_ODBC_LOB_SIZE_ALLOWED).set(operation.get(Configuration.MAX_ODBC_LOB_SIZE_ALLOWED).asInt());
- }
- if (operation.hasDefined(Configuration.SECURITY_DOMAIN)) {
- List<ModelNode> domains = operation.get(Configuration.SECURITY_DOMAIN).asList();
- for (ModelNode domain: domains) {
- model.get(Configuration.SECURITY_DOMAIN).add(domain.asString());
- }
- }
- if (operation.hasDefined(Configuration.DETECTING_CHANGE_EVENTS)) {
- model.get(Configuration.DETECTING_CHANGE_EVENTS).set(operation.get(Configuration.DETECTING_CHANGE_EVENTS).asBoolean());
- }
- if (operation.hasDefined(Configuration.SESSION_EXPIRATION_TIME_LIMIT)) {
- model.get(Configuration.SESSION_EXPIRATION_TIME_LIMIT).set(operation.get(Configuration.SESSION_EXPIRATION_TIME_LIMIT).asInt());
- }
- if (operation.hasDefined(Configuration.MAX_SESSIONS_ALLOWED)) {
- model.get(Configuration.MAX_SESSIONS_ALLOWED).set(operation.get(Configuration.MAX_SESSIONS_ALLOWED).asInt());
- }
-
- populateSocketConfiguration(Configuration.JDBC+TeiidBootServicesAdd.DASH, operation, model);
-
- populateSocketConfiguration(Configuration.ODBC+TeiidBootServicesAdd.DASH, operation, model);
- }
-
- private static void populateSocketConfiguration(String prefix, ModelNode operation, ModelNode model) {
- if (operation.hasDefined(prefix+Configuration.SOCKET_BINDING)) {
- model.get(prefix+Configuration.SOCKET_BINDING).set(operation.get(prefix+Configuration.SOCKET_BINDING).asString());
- }
- if (operation.hasDefined(prefix+Configuration.MAX_SOCKET_THREAD_SIZE)) {
- model.get(prefix+Configuration.MAX_SOCKET_THREAD_SIZE).set(operation.get(prefix+Configuration.MAX_SOCKET_THREAD_SIZE).asInt());
- }
- if (operation.hasDefined(prefix+Configuration.IN_BUFFER_SIZE)) {
- model.get(prefix+Configuration.IN_BUFFER_SIZE).set(operation.get(prefix+Configuration.IN_BUFFER_SIZE).asInt());
- }
- if (operation.hasDefined(prefix+Configuration.OUT_BUFFER_SIZE)) {
- model.get(prefix+Configuration.OUT_BUFFER_SIZE).set(operation.get(prefix+Configuration.OUT_BUFFER_SIZE).asInt());
- }
-
- String sslPrefix = prefix+Configuration.SSL+TeiidBootServicesAdd.DASH;
-
- if (operation.hasDefined(sslPrefix+Configuration.SSL_MODE)) {
- model.get(sslPrefix+Configuration.SSL_MODE).set(operation.get(sslPrefix+Configuration.SSL_MODE).asString());
- }
-
- if (operation.hasDefined(sslPrefix+Configuration.KEY_STORE_FILE)) {
- model.get(sslPrefix+Configuration.KEY_STORE_FILE).set(operation.get(sslPrefix+Configuration.KEY_STORE_FILE).asString());
- }
-
- if (operation.hasDefined(sslPrefix+Configuration.KEY_STORE_PASSWD)) {
- model.get(sslPrefix+Configuration.KEY_STORE_PASSWD).set(operation.get(sslPrefix+Configuration.KEY_STORE_PASSWD).asString());
- }
-
- if (operation.hasDefined(sslPrefix+Configuration.KEY_STORE_TYPE)) {
- model.get(sslPrefix+Configuration.KEY_STORE_TYPE).set(operation.get(sslPrefix+Configuration.KEY_STORE_TYPE).asString());
- }
-
- if (operation.hasDefined(sslPrefix+Configuration.SSL_PROTOCOL)) {
- model.get(sslPrefix+Configuration.SSL_PROTOCOL).set(operation.get(sslPrefix+Configuration.SSL_PROTOCOL).asString());
- }
- if (operation.hasDefined(sslPrefix+Configuration.KEY_MANAGEMENT_ALG)) {
- model.get(sslPrefix+Configuration.KEY_MANAGEMENT_ALG).set(operation.get(sslPrefix+Configuration.KEY_MANAGEMENT_ALG).asString());
- }
- if (operation.hasDefined(sslPrefix+Configuration.TRUST_FILE)) {
- model.get(sslPrefix+Configuration.TRUST_FILE).set(operation.get(sslPrefix+Configuration.TRUST_FILE).asString());
- }
- if (operation.hasDefined(sslPrefix+Configuration.TRUST_PASSWD)) {
- model.get(sslPrefix+Configuration.TRUST_PASSWD).set(operation.get(sslPrefix+Configuration.TRUST_PASSWD).asString());
- }
- if (operation.hasDefined(sslPrefix+Configuration.AUTH_MODE)) {
- model.get(sslPrefix+Configuration.AUTH_MODE).set(operation.get(sslPrefix+Configuration.AUTH_MODE).asString());
- }
- }
-}
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineOperationHandler.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineOperationHandler.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineOperationHandler.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -21,12 +21,27 @@
*/
package org.teiid.jboss;
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REPLY_PROPERTIES;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REQUEST_PROPERTIES;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REQUIRED;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.TYPE;
import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.SQLException;
+import java.sql.SQLXML;
+import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.ResourceBundle;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
import javax.xml.stream.XMLStreamException;
@@ -36,31 +51,57 @@
import org.jboss.dmr.ModelNode;
import org.jboss.dmr.ModelType;
import org.jboss.msc.service.ServiceController;
+import org.jboss.msc.service.ServiceName;
import org.teiid.adminapi.Admin;
import org.teiid.adminapi.AdminException;
import org.teiid.adminapi.AdminProcessingException;
import org.teiid.adminapi.impl.*;
import org.teiid.adminapi.impl.VDBMetadataMapper.TransactionMetadataMapper;
import org.teiid.adminapi.impl.VDBMetadataMapper.VDBTranslatorMetaDataMapper;
+import org.teiid.client.RequestMessage;
+import org.teiid.client.ResultsMessage;
+import org.teiid.client.security.SessionToken;
+import org.teiid.client.util.ResultsFuture;
+import org.teiid.core.TeiidComponentException;
import org.teiid.deployers.VDBRepository;
import org.teiid.dqp.internal.datamgr.TranslatorRepository;
+import org.teiid.dqp.internal.process.DQPCore;
+import org.teiid.dqp.internal.process.DQPWorkContext;
import org.teiid.dqp.internal.process.SessionAwareCache;
-import org.teiid.jboss.deployers.RuntimeEngineDeployer;
import org.teiid.logging.LogConstants;
import org.teiid.logging.LogManager;
-abstract class QueryEngineOperationHandler extends BaseOperationHandler<RuntimeEngineDeployer> {
+abstract class QueryEngineOperationHandler extends BaseOperationHandler<DQPCore> {
+ List<Transport> transports = new ArrayList<Transport>();
+ protected VDBRepository vdbRepo;
+ protected DQPCore engine;
protected QueryEngineOperationHandler(String operationName){
super(operationName);
}
@Override
- protected RuntimeEngineDeployer getService(OperationContext context, PathAddress pathAddress, ModelNode operation) throws OperationFailedException {
- String serviceName = pathAddress.getLastElement().getValue();
- ServiceController<?> sc = context.getServiceRegistry(false).getRequiredService(TeiidServiceNames.engineServiceName(serviceName));
- return RuntimeEngineDeployer.class.cast(sc.getValue());
- }
+ protected DQPCore getService(OperationContext context, PathAddress pathAddress, ModelNode operation) throws OperationFailedException {
+ List<ServiceName> services = context.getServiceRegistry(false).getServiceNames();
+ for (ServiceName name:services) {
+ if (name.isParentOf(TeiidServiceNames.TRANSPORT_BASE)) {
+ ServiceController<?> transport = context.getServiceRegistry(false).getService(name);
+ if (transport != null) {
+ this.transports.add(Transport.class.cast(transport.getValue()));
+ }
+ }
+ }
+ ServiceController<?> repo = context.getServiceRegistry(false).getRequiredService(TeiidServiceNames.VDB_REPO);
+ if (repo != null) {
+ this.vdbRepo = VDBRepository.class.cast(repo.getValue());
+ }
+
+ ServiceController<?> sc = context.getServiceRegistry(false).getRequiredService(TeiidServiceNames.ENGINE);
+ if (sc != null) {
+ this.engine = DQPCore.class.cast(sc.getValue());
+ }
+ return this.engine;
+ }
}
abstract class TranslatorOperationHandler extends BaseOperationHandler<TranslatorRepository> {
@@ -81,7 +122,7 @@
super(operationName);
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
context.getResult().set(engine.getRuntimeVersion());
}
protected void describeParameters(ModelNode operationNode, ResourceBundle bundle) {
@@ -94,9 +135,13 @@
super(operationName);
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
try {
- context.getResult().set(String.valueOf(engine.getActiveSessionsCount()));
+ int count = 0;
+ for (Transport t: this.transports) {
+ count += t.getActiveSessionsCount();
+ }
+ context.getResult().set(String.valueOf(count));
} catch (AdminException e) {
throw new OperationFailedException(new ModelNode().set(e.getMessage()));
}
@@ -111,13 +156,15 @@
super("list-sessions"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
try {
ModelNode result = context.getResult();
- Collection<SessionMetadata> sessions = engine.getActiveSessions();
- for (SessionMetadata session:sessions) {
- VDBMetadataMapper.SessionMetadataMapper.INSTANCE.wrap(session, result.add());
- }
+ for (Transport t: this.transports) {
+ Collection<SessionMetadata> sessions = t.getActiveSessions();
+ for (SessionMetadata session:sessions) {
+ VDBMetadataMapper.SessionMetadataMapper.INSTANCE.wrap(session, result.add());
+ }
+ }
} catch (AdminException e) {
throw new OperationFailedException(new ModelNode().set(e.getMessage()));
}
@@ -133,7 +180,7 @@
super("requests-per-session"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
if (!operation.hasDefined(OperationsConstants.SESSION)) {
throw new OperationFailedException(new ModelNode().set(IntegrationPlugin.Util.getString(OperationsConstants.SESSION+MISSING)));
}
@@ -158,7 +205,7 @@
super("list-requests"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
ModelNode result = context.getResult();
List<RequestMetadata> requests = engine.getRequests();
for (RequestMetadata request:requests) {
@@ -175,7 +222,7 @@
super("requests-per-vdb"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
try {
if (!operation.hasDefined(OperationsConstants.VDB_NAME)) {
@@ -188,9 +235,11 @@
ModelNode result = context.getResult();
String vdbName = operation.get(OperationsConstants.VDB_NAME).asString();
int vdbVersion = operation.get(OperationsConstants.VDB_VERSION).asInt();
- List<RequestMetadata> requests = engine.getRequestsUsingVDB(vdbName,vdbVersion);
- for (RequestMetadata request:requests) {
- VDBMetadataMapper.RequestMetadataMapper.INSTANCE.wrap(request, result.add());
+ for (Transport t: this.transports) {
+ List<RequestMetadata> requests = t.getRequestsUsingVDB(vdbName,vdbVersion);
+ for (RequestMetadata request:requests) {
+ VDBMetadataMapper.RequestMetadataMapper.INSTANCE.wrap(request, result.add());
+ }
}
} catch (AdminException e) {
throw new OperationFailedException(e, new ModelNode().set(e.getMessage()));
@@ -215,7 +264,7 @@
super("long-running-queries"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
ModelNode result = context.getResult();
List<RequestMetadata> requests = engine.getLongRunningRequests();
for (RequestMetadata request:requests) {
@@ -232,11 +281,13 @@
super("terminate-session"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
if (!operation.hasDefined(OperationsConstants.SESSION)) {
throw new OperationFailedException(new ModelNode().set(IntegrationPlugin.Util.getString(OperationsConstants.SESSION+MISSING)));
}
- engine.terminateSession(operation.get(OperationsConstants.SESSION).asString());
+ for (Transport t: this.transports) {
+ t.terminateSession(operation.get(OperationsConstants.SESSION).asString());
+ }
}
protected void describeParameters(ModelNode operationNode, ResourceBundle bundle) {
@@ -251,7 +302,7 @@
super("cancel-request"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException{
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException{
try {
if (!operation.hasDefined(OperationsConstants.SESSION)) {
throw new OperationFailedException(new ModelNode().set(IntegrationPlugin.Util.getString(OperationsConstants.SESSION+MISSING)));
@@ -262,7 +313,7 @@
boolean pass = engine.cancelRequest(operation.get(OperationsConstants.SESSION).asString(), operation.get(OperationsConstants.EXECUTION_ID).asLong());
ModelNode result = context.getResult();
result.set(pass);
- } catch (AdminException e) {
+ } catch (TeiidComponentException e) {
throw new OperationFailedException(e, new ModelNode().set(e.getMessage()));
}
}
@@ -413,7 +464,7 @@
super("workerpool-statistics"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException {
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException {
ModelNode result = context.getResult();
WorkerPoolStatisticsMetadata stats = engine.getWorkerPoolStatistics();
VDBMetadataMapper.WorkerPoolStatisticsMetadataMapper.INSTANCE.wrap(stats, result);
@@ -429,7 +480,7 @@
super("list-transactions"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException {
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException {
ModelNode result = context.getResult();
Collection<TransactionMetadata> txns = engine.getTransactions();
for (TransactionMetadata txn:txns) {
@@ -448,7 +499,7 @@
super("terminate-transaction"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException {
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException {
if (!operation.hasDefined(OperationsConstants.XID)) {
throw new OperationFailedException(new ModelNode().set(IntegrationPlugin.Util.getString(OperationsConstants.XID+MISSING)));
@@ -533,7 +584,7 @@
super("execute-query"); //$NON-NLS-1$
}
@Override
- protected void executeOperation(OperationContext context, RuntimeEngineDeployer engine, ModelNode operation) throws OperationFailedException {
+ protected void executeOperation(OperationContext context, DQPCore engine, ModelNode operation) throws OperationFailedException {
if (!operation.hasDefined(OperationsConstants.VDB_NAME)) {
throw new OperationFailedException(new ModelNode().set(IntegrationPlugin.Util.getString(OperationsConstants.VDB_NAME+MISSING)));
@@ -553,26 +604,10 @@
int vdbVersion = operation.get(OperationsConstants.VDB_VERSION).asInt();
String sql = operation.get(OperationsConstants.SQL_QUERY).asString();
int timeout = operation.get(OperationsConstants.TIMEOUT_IN_MILLI).asInt();
- try {
- List<List> results = engine.executeQuery(vdbName, vdbVersion, sql, timeout);
- List colNames = results.get(0);
- for (int rowNum = 1; rowNum < results.size(); rowNum++) {
-
- List row = results.get(rowNum);
- ModelNode rowNode = new ModelNode();
- rowNode.get(TYPE).set(ModelType.OBJECT);
-
- for (int colNum = 0; colNum < colNames.size(); colNum++) {
- //TODO: support in native types instead of string here.
- rowNode.get(ATTRIBUTES, colNames.get(colNum).toString()).set(row.get(colNum).toString());
- }
- result.add(rowNode);
- }
- } catch (AdminException e) {
- throw new OperationFailedException(new ModelNode().set(e.getMessage()));
- }
+
+ result.set(executeQuery(vdbName, vdbVersion, sql, timeout, new ModelNode()));
}
-
+
protected void describeParameters(ModelNode operationNode, ResourceBundle bundle) {
operationNode.get(REQUEST_PROPERTIES, OperationsConstants.VDB_NAME, TYPE).set(ModelType.STRING);
operationNode.get(REQUEST_PROPERTIES, OperationsConstants.VDB_NAME, REQUIRED).set(true);
@@ -592,6 +627,133 @@
operationNode.get(REPLY_PROPERTIES).set(ModelType.LIST);
}
+
+ public ModelNode executeQuery(final String vdbName, final int version, final String command, final long timoutInMilli, final ModelNode resultsNode) throws OperationFailedException {
+ String user = "CLI ADMIN"; //$NON-NLS-1$
+ LogManager.logDetail(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("admin_executing", user, command)); //$NON-NLS-1$
+
+ SessionMetadata session = createTemporarySession(vdbName, version, user);
+
+ final long requestID = 0L;
+
+ DQPWorkContext context = new DQPWorkContext();
+ context.setSession(session);
+
+ try {
+ return context.runInContext(new Callable<ModelNode>() {
+ @Override
+ public ModelNode call() throws Exception {
+
+ long start = System.currentTimeMillis();
+ RequestMessage request = new RequestMessage(command);
+ request.setExecutionId(0L);
+ request.setRowLimit(engine.getMaxRowsFetchSize()); // this would limit the number of rows that are returned.
+ Future<ResultsMessage> message = engine.executeRequest(requestID, request);
+ ResultsMessage rm = null;
+ if (timoutInMilli < 0) {
+ rm = message.get();
+ } else {
+ rm = message.get(timoutInMilli, TimeUnit.MILLISECONDS);
+ }
+ if (rm.getException() != null) {
+ throw new AdminProcessingException(rm.getException());
+ }
+
+ if (rm.isUpdateResult()) {
+ writeResults(resultsNode, Arrays.asList("update-count"), rm.getResultsList()); //$NON-NLS-1$
+ }
+ else {
+ writeResults(resultsNode, Arrays.asList(rm.getColumnNames()), rm.getResultsList());
+
+ while (rm.getFinalRow() == -1 || rm.getLastRow() < rm.getFinalRow()) {
+ long elapsed = System.currentTimeMillis() - start;
+ message = engine.processCursorRequest(requestID, rm.getLastRow()+1, 1024);
+ rm = message.get(timoutInMilli-elapsed, TimeUnit.MILLISECONDS);
+ writeResults(resultsNode, Arrays.asList(rm.getColumnNames()), rm.getResultsList());
+ }
+ }
+
+ long elapsed = System.currentTimeMillis() - start;
+ ResultsFuture<?> response = engine.closeRequest(requestID);
+ response.get(timoutInMilli-elapsed, TimeUnit.MILLISECONDS);
+ return resultsNode;
+ }
+ });
+ } catch (Throwable t) {
+ throw new OperationFailedException(new ModelNode().set(t.getMessage()));
+ }
+ }
+
+ private void writeResults(ModelNode resultsNode, List<String> columns, List<? extends List<?>> results) throws SQLException {
+ for (List<?> row:results) {
+ ModelNode rowNode = new ModelNode();
+
+ for (int colNum = 0; colNum < columns.size(); colNum++) {
+
+ Object aValue = row.get(colNum);
+ if (aValue != null) {
+ if (aValue instanceof Integer) {
+ rowNode.get(columns.get(colNum)).set((Integer)aValue);
+ }
+ else if (aValue instanceof Long) {
+ rowNode.get(columns.get(colNum)).set((Long)aValue);
+ }
+ else if (aValue instanceof Double) {
+ rowNode.get(columns.get(colNum)).set((Double)aValue);
+ }
+ else if (aValue instanceof Boolean) {
+ rowNode.get(columns.get(colNum)).set((Boolean)aValue);
+ }
+ else if (aValue instanceof BigInteger) {
+ rowNode.get(columns.get(colNum)).set((BigInteger)aValue);
+ }
+ else if (aValue instanceof BigDecimal) {
+ rowNode.get(columns.get(colNum)).set((BigDecimal)aValue);
+ }
+ else if (aValue instanceof String) {
+ rowNode.get(columns.get(colNum), TYPE).set(ModelType.STRING);
+ rowNode.get(columns.get(colNum)).set((String)aValue);
+ }
+ else if (aValue instanceof Blob) {
+ rowNode.get(columns.get(colNum), TYPE).set(ModelType.OBJECT);
+ rowNode.get(columns.get(colNum)).set("blob"); //$NON-NLS-1$
+ }
+ else if (aValue instanceof Clob) {
+ rowNode.get(columns.get(colNum), TYPE).set(ModelType.OBJECT);
+ rowNode.get(columns.get(colNum)).set("clob"); //$NON-NLS-1$
+ }
+ else if (aValue instanceof SQLXML) {
+ SQLXML xml = (SQLXML)aValue;
+ rowNode.get(columns.get(colNum), TYPE).set(ModelType.STRING);
+ rowNode.get(columns.get(colNum)).set(xml.getString());
+ }
+ else {
+ rowNode.get(columns.get(colNum), TYPE).set(ModelType.STRING);
+ rowNode.get(columns.get(colNum)).set(aValue.toString());
+ }
+ }
+ }
+ resultsNode.add(rowNode);
+ }
+ }
+
+ private SessionMetadata createTemporarySession(final String vdbName, final int version, final String userName) {
+
+ long creationTime = System.currentTimeMillis();
+
+ // Return a new session info object
+ SessionMetadata newSession = new SessionMetadata();
+ newSession.setSessionToken(new SessionToken(userName));
+ newSession.setSessionId(newSession.getSessionToken().getSessionID());
+ newSession.setUserName(userName);
+ newSession.setCreatedTime(creationTime);
+ newSession.setApplicationName("admin-console"); //$NON-NLS-1$
+ newSession.setVDBName(vdbName);
+ newSession.setVDBVersion(version);
+
+ newSession.setVdb(this.vdbRepo.getVDB(vdbName, version));
+ return newSession;
+ }
}
class GetVDB extends BaseOperationHandler<VDBRepository>{
Deleted: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineReferenceFactoryService.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineReferenceFactoryService.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineReferenceFactoryService.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,62 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * Copyright 2011, Red Hat, Inc., and individual contributors
- * as indicated by the @author tags. See the copyright.txt file in the
- * distribution for a full listing of individual contributors.
- *
- * This is free software; you can redistribute it and/or modify it
- * under the terms of the GNU Lesser General Public License as
- * published by the Free Software Foundation; either version 2.1 of
- * the License, or (at your option) any later version.
- *
- * This software is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this software; if not, write to the Free
- * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
- * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
- */
-
-package org.teiid.jboss;
-
-import org.jboss.as.naming.ManagedReference;
-import org.jboss.as.naming.ManagedReferenceFactory;
-import org.jboss.as.naming.ValueManagedReference;
-import org.jboss.msc.inject.Injector;
-import org.jboss.msc.service.Service;
-import org.jboss.msc.service.StartContext;
-import org.jboss.msc.service.StartException;
-import org.jboss.msc.service.StopContext;
-import org.jboss.msc.value.ImmediateValue;
-import org.jboss.msc.value.InjectedValue;
-import org.teiid.jboss.deployers.RuntimeEngineDeployer;
-
-
-class QueryEngineReferenceFactoryService implements Service<ManagedReferenceFactory>, ManagedReferenceFactory {
- private final InjectedValue<RuntimeEngineDeployer> engineInjector = new InjectedValue<RuntimeEngineDeployer>();
-
- private ManagedReference reference;
-
- public synchronized void start(StartContext startContext) throws StartException {
- reference = new ValueManagedReference(new ImmediateValue<Object>(engineInjector.getValue()));
- }
-
- public synchronized void stop(StopContext stopContext) {
- reference = null;
- }
-
- public synchronized ManagedReferenceFactory getValue() throws IllegalStateException, IllegalArgumentException {
- return this;
- }
-
- public synchronized ManagedReference getReference() {
- return reference;
- }
-
- public Injector<RuntimeEngineDeployer> getQueryEngineInjector() {
- return engineInjector;
- }
-}
Deleted: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineRemove.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineRemove.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineRemove.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,79 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership. Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- *
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- *
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-package org.teiid.jboss;
-
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-
-import java.util.Locale;
-import java.util.ResourceBundle;
-
-import org.jboss.as.controller.AbstractRemoveStepHandler;
-import org.jboss.as.controller.OperationContext;
-import org.jboss.as.controller.PathAddress;
-import org.jboss.as.controller.descriptions.DescriptionProvider;
-import org.jboss.as.naming.deployment.ContextNames;
-import org.jboss.dmr.ModelNode;
-import org.jboss.msc.service.ServiceController;
-import org.jboss.msc.service.ServiceName;
-import org.jboss.msc.service.ServiceRegistry;
-import org.teiid.transport.LocalServerConnection;
-
-class QueryEngineRemove extends AbstractRemoveStepHandler implements DescriptionProvider {
-
- @Override
- protected void performRuntime(OperationContext context, ModelNode operation, ModelNode model) {
-
- final ModelNode address = operation.require(OP_ADDR);
- final PathAddress pathAddress = PathAddress.pathAddress(address);
-
- String engineName = pathAddress.getLastElement().getValue();
-
- final ServiceRegistry serviceRegistry = context.getServiceRegistry(true);
- ServiceName serviceName = TeiidServiceNames.engineServiceName(engineName);
- final ServiceController<?> controller = serviceRegistry.getService(serviceName);
- if (controller != null) {
- context.removeService(serviceName);
- }
-
- final ServiceName referenceFactoryServiceName = TeiidServiceNames.engineServiceName(engineName).append("reference-factory"); //$NON-NLS-1$
- final ServiceController<?> referceFactoryController = serviceRegistry.getService(referenceFactoryServiceName);
- if (referceFactoryController != null) {
- context.removeService(referenceFactoryServiceName);
- }
-
- final ContextNames.BindInfo bindInfo = ContextNames.bindInfoFor(LocalServerConnection.TEIID_RUNTIME_CONTEXT+engineName);
- final ServiceController<?> binderController = serviceRegistry.getService(bindInfo.getBinderServiceName());
- if (binderController != null) {
- context.removeService(bindInfo.getBinderServiceName());
- }
- }
-
- @Override
- public ModelNode getModelDescription(Locale locale) {
- final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
- final ModelNode operation = new ModelNode();
- operation.get(OPERATION_NAME).set(REMOVE);
- operation.get(DESCRIPTION).set(bundle.getString(REMOVE+"."+DESCRIBE)); //$NON-NLS-1$
- return operation;
- }
-
-}
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidBootServicesAdd.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidBootServicesAdd.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidBootServicesAdd.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -22,15 +22,20 @@
package org.teiid.jboss;
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-import static org.teiid.jboss.Configuration.DESC;
-import static org.teiid.jboss.Configuration.addAttribute;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.ADD;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OPERATION_NAME;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REQUEST_PROPERTIES;
import java.util.List;
import java.util.Locale;
import java.util.ResourceBundle;
import java.util.ServiceLoader;
+import javax.resource.spi.XATerminator;
+import javax.resource.spi.work.WorkManager;
+import javax.transaction.TransactionManager;
+
import org.jboss.as.clustering.jgroups.ChannelFactory;
import org.jboss.as.controller.AbstractAddStepHandler;
import org.jboss.as.controller.OperationContext;
@@ -42,11 +47,17 @@
import org.jboss.as.server.deployment.Phase;
import org.jboss.as.server.services.path.RelativePathService;
import org.jboss.dmr.ModelNode;
-import org.jboss.dmr.ModelType;
import org.jboss.modules.Module;
import org.jboss.modules.ModuleIdentifier;
import org.jboss.modules.ModuleLoadException;
-import org.jboss.msc.service.*;
+import org.jboss.msc.service.ServiceBuilder;
+import org.jboss.msc.service.ServiceBuilder.DependencyType;
+import org.jboss.msc.service.ServiceContainer;
+import org.jboss.msc.service.ServiceController;
+import org.jboss.msc.service.ServiceName;
+import org.jboss.msc.service.ServiceTarget;
+import org.jboss.msc.service.ValueService;
+import org.jboss.msc.value.InjectedValue;
import org.teiid.PolicyDecider;
import org.teiid.cache.CacheConfiguration;
import org.teiid.cache.CacheConfiguration.Policy;
@@ -56,77 +67,103 @@
import org.teiid.deployers.SystemVDBDeployer;
import org.teiid.deployers.VDBRepository;
import org.teiid.dqp.internal.datamgr.TranslatorRepository;
-import org.teiid.dqp.internal.process.*;
+import org.teiid.dqp.internal.process.AuthorizationValidator;
+import org.teiid.dqp.internal.process.CachedResults;
+import org.teiid.dqp.internal.process.DQPCore;
+import org.teiid.dqp.internal.process.DataRolePolicyDecider;
+import org.teiid.dqp.internal.process.DefaultAuthorizationValidator;
+import org.teiid.dqp.internal.process.PreparedPlan;
+import org.teiid.dqp.internal.process.SessionAwareCache;
+import org.teiid.jboss.deployers.RuntimeEngineDeployer;
+import org.teiid.query.ObjectReplicator;
import org.teiid.query.function.SystemFunctionManager;
import org.teiid.replication.jboss.JGroupsObjectReplicator;
import org.teiid.services.BufferServiceImpl;
class TeiidBootServicesAdd extends AbstractAddStepHandler implements DescriptionProvider {
- static final String DASH = "-"; //$NON-NLS-1$
+ private static Element[] attributes = {
+ Element.ALLOW_ENV_FUNCTION_ELEMENT,
+ Element.ASYNC_THREAD_POOL_ELEMENT,
+ Element.MAX_THREADS_ELEMENT,
+ Element.MAX_ACTIVE_PLANS_ELEMENT,
+ Element.USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT,
+ Element.TIME_SLICE_IN_MILLI_ELEMENT,
+ Element.MAX_ROWS_FETCH_SIZE_ELEMENT,
+ Element.LOB_CHUNK_SIZE_IN_KB_ELEMENT,
+ Element.QUERY_THRESHOLD_IN_SECS_ELEMENT,
+ Element.MAX_SOURCE_ROWS_ELEMENT,
+ Element.EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT,
+ Element.DETECTING_CHANGE_EVENTS_ELEMENT,
+ Element.AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE,
+ Element.POLICY_DECIDER_MODULE_ATTRIBUTE,
+
+ // object replicator
+ Element.OR_STACK_ATTRIBUTE,
+ Element.OR_CLUSTER_NAME_ATTRIBUTE,
+ // Buffer Service
+ Element.USE_DISK_ATTRIBUTE,
+ Element.PROCESSOR_BATCH_SIZE_ATTRIBUTE,
+ Element.CONNECTOR_BATCH_SIZE_ATTRIBUTE,
+ Element.MAX_PROCESSING_KB_ATTRIBUTE,
+ Element.MAX_RESERVED_KB_ATTRIBUTE,
+ Element.MAX_FILE_SIZE_ATTRIBUTE,
+ Element.MAX_BUFFER_SPACE_ATTRIBUTE,
+ Element.MAX_OPEN_FILES_ATTRIBUTE,
+
+ // prepared plan cache
+ Element.PPC_MAX_ENTRIES_ATTRIBUTE,
+ Element.PPC_MAX_AGE_IN_SECS_ATTRIBUTE,
+ Element.PPC_MAX_STALENESS_ATTRIBUTE,
+
+ // resultset cache
+ Element.RSC_NAME_ELEMENT,
+ Element.RSC_CONTAINER_NAME_ELEMENT,
+ Element.RSC_MAX_STALENESS_ELEMENT,
+ Element.RSC_ENABLE_ATTRIBUTE
+ };
+
@Override
public ModelNode getModelDescription(Locale locale) {
final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
final ModelNode node = new ModelNode();
node.get(OPERATION_NAME).set(ADD);
- node.get(DESCRIPTION).set(bundle.getString("teiid-boot.add")); //$NON-NLS-1$
+ node.get(DESCRIPTION).set(bundle.getString("teiid.add")); //$NON-NLS-1$
- describeTeiidRoot(bundle, REQUEST_PROPERTIES, node);
+ describeTeiid(node, REQUEST_PROPERTIES, bundle);
return node;
}
- static void describeTeiidRoot(final ResourceBundle bundle, String type, final ModelNode node) {
- addAttribute(node, Configuration.ALLOW_ENV_FUNCTION, type, bundle.getString(Configuration.ALLOW_ENV_FUNCTION+DESC), ModelType.BOOLEAN, false, "false"); //$NON-NLS-1$
- addAttribute(node, Configuration.ASYNC_THREAD_GROUP, type, bundle.getString(Configuration.ASYNC_THREAD_GROUP+DESC), ModelType.STRING, true, null);
-
- addAttribute(node, Configuration.AUTHORIZATION_VALIDATOR_MODULE, type, bundle.getString(Configuration.AUTHORIZATION_VALIDATOR_MODULE+DESC), ModelType.BOOLEAN, false, null);
- addAttribute(node, Configuration.POLICY_DECIDER_MODULE, type, bundle.getString(Configuration.POLICY_DECIDER_MODULE+DESC), ModelType.STRING, false, null);
-
- describeObjectReplicator(node, type, bundle);
- describeBufferManager(node, type, bundle);
- describePreparedPlanCache(node, type, bundle);
- describeResultsetCache(node, type, bundle);
+ static void describeTeiid(final ModelNode node, String type, final ResourceBundle bundle) {
+ for (int i = 0; i < attributes.length; i++) {
+ attributes[i].describe(node, type, bundle);
+ }
}
-
+
@Override
protected void populateModel(ModelNode operation, ModelNode model) throws OperationFailedException {
populate(operation, model);
}
static void populate(ModelNode operation, ModelNode model) {
- if (operation.hasDefined(Configuration.ALLOW_ENV_FUNCTION)) {
- model.get(Configuration.ALLOW_ENV_FUNCTION).set(operation.get(Configuration.ALLOW_ENV_FUNCTION).asString());
+ for (int i = 0; i < attributes.length; i++) {
+ attributes[i].populate(operation, model);
}
- if (operation.hasDefined(Configuration.ASYNC_THREAD_GROUP)) {
- model.get(Configuration.ASYNC_THREAD_GROUP).set(operation.get(Configuration.ASYNC_THREAD_GROUP).asString());
- }
-
- populateBufferManager(operation, model);
-
- if (operation.hasDefined(Configuration.POLICY_DECIDER_MODULE)) {
- model.get(Configuration.POLICY_DECIDER_MODULE).set(operation.get(Configuration.POLICY_DECIDER_MODULE).asString());
- }
-
- if (operation.hasDefined(Configuration.AUTHORIZATION_VALIDATOR_MODULE)) {
- model.get(Configuration.AUTHORIZATION_VALIDATOR_MODULE).set(operation.get(Configuration.AUTHORIZATION_VALIDATOR_MODULE).asString());
- }
-
- populateResultsetCache(operation, model);
- populatePreparedPlanCache(operation, model);
- populateObjectReplicator(operation, model);
}
-
@Override
protected void performRuntime(final OperationContext context, final ModelNode operation, final ModelNode model,
final ServiceVerificationHandler verificationHandler, final List<ServiceController<?>> newControllers) throws OperationFailedException {
+
ServiceTarget target = context.getServiceTarget();
- final String asyncThreadPoolName = operation.require(Configuration.ASYNC_THREAD_GROUP).asString();
+ final JBossLifeCycleListener shutdownListener = new JBossLifeCycleListener();
+ final String asyncThreadPoolName = Element.ASYNC_THREAD_POOL_ELEMENT.asString(operation);
+
// translator repository
final TranslatorRepository translatorRepo = new TranslatorRepository();
ValueService<TranslatorRepository> translatorService = new ValueService<TranslatorRepository>(new org.jboss.msc.value.Value<TranslatorRepository>() {
@@ -140,8 +177,8 @@
// system function tree
SystemFunctionManager systemFunctionManager = new SystemFunctionManager();
- if (operation.hasDefined(Configuration.ALLOW_ENV_FUNCTION)) {
- systemFunctionManager.setAllowEnvFunction(operation.get(Configuration.ALLOW_ENV_FUNCTION).asBoolean());
+ if (Element.ALLOW_ENV_FUNCTION_ELEMENT.isDefined(operation)) {
+ systemFunctionManager.setAllowEnvFunction(Element.ALLOW_ENV_FUNCTION_ELEMENT.asBoolean(operation));
}
else {
systemFunctionManager.setAllowEnvFunction(false);
@@ -175,8 +212,8 @@
newControllers.add(bufferServiceBuilder.install());
PolicyDecider policyDecider;
- if (operation.hasDefined(Configuration.POLICY_DECIDER_MODULE)) {
- policyDecider = buildService(PolicyDecider.class, operation.get(Configuration.POLICY_DECIDER_MODULE).asString());
+ if (Element.POLICY_DECIDER_MODULE_ATTRIBUTE.isDefined(operation)) {
+ policyDecider = buildService(PolicyDecider.class, Element.POLICY_DECIDER_MODULE_ATTRIBUTE.asString(operation));
}
else {
DataRolePolicyDecider drpd = new DataRolePolicyDecider();
@@ -186,8 +223,8 @@
}
final AuthorizationValidator authValidator;
- if (operation.hasDefined(Configuration.AUTHORIZATION_VALIDATOR_MODULE)) {
- authValidator = buildService(AuthorizationValidator.class, operation.get(Configuration.AUTHORIZATION_VALIDATOR_MODULE).asString());
+ if (Element.AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE.isDefined(operation)) {
+ authValidator = buildService(AuthorizationValidator.class, Element.AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE.asString(operation));
authValidator.setEnabled(true);
}
else {
@@ -226,12 +263,14 @@
newControllers.add(target.addService(TeiidServiceNames.CACHE_PREPAREDPLAN, preparedPlanService).install());
// Object Replicator
- if (operation.hasDefined(ORC(Configuration.STACK))) {
- String stack = operation.get(ORC(Configuration.STACK)).asString();
+ if (Element.OR_STACK_ATTRIBUTE.isDefined(operation)) {
+ String stack = Element.OR_STACK_ATTRIBUTE.asString(operation);
+
String clusterName = "teiid-rep"; //$NON-NLS-1$
- if (operation.hasDefined(ORC(Configuration.CLUSTER_NAME))) {
- clusterName = operation.get(ORC(Configuration.CLUSTER_NAME)).asString();
+ if (Element.OR_CLUSTER_NAME_ATTRIBUTE.isDefined(operation)) {
+ clusterName = Element.OR_CLUSTER_NAME_ATTRIBUTE.asString(operation);
}
+
JGroupsObjectReplicatorService replicatorService = new JGroupsObjectReplicatorService(clusterName);
replicatorService.setBufferManager(bufferManager.getBufferManager());
ServiceBuilder<JGroupsObjectReplicator> serviceBuilder = target.addService(TeiidServiceNames.OBJECT_REPLICATOR, replicatorService);
@@ -239,7 +278,28 @@
newControllers.add(serviceBuilder.install());
}
+ // Query Engine
+ final RuntimeEngineDeployer engine = buildQueryEngine(operation);
+ ServiceBuilder<DQPCore> engineBuilder = target.addService(TeiidServiceNames.ENGINE, engine);
+ engineBuilder.addDependency(ServiceName.JBOSS.append("connector", "workmanager"), WorkManager.class, engine.getWorkManagerInjector()); //$NON-NLS-1$ //$NON-NLS-2$
+ engineBuilder.addDependency(ServiceName.JBOSS.append("txn", "XATerminator"), XATerminator.class, engine.getXaTerminatorInjector()); //$NON-NLS-1$ //$NON-NLS-2$
+ engineBuilder.addDependency(ServiceName.JBOSS.append("txn", "TransactionManager"), TransactionManager.class, engine.getTxnManagerInjector()); //$NON-NLS-1$ //$NON-NLS-2$
+ engineBuilder.addDependency(TeiidServiceNames.BUFFER_MGR, BufferServiceImpl.class, engine.getBufferServiceInjector());
+ engineBuilder.addDependency(TeiidServiceNames.SYSTEM_VDB, SystemVDBDeployer.class, new InjectedValue<SystemVDBDeployer>());
+ engineBuilder.addDependency(TeiidServiceNames.TRANSLATOR_REPO, TranslatorRepository.class, engine.getTranslatorRepositoryInjector());
+ engineBuilder.addDependency(TeiidServiceNames.VDB_REPO, VDBRepository.class, engine.getVdbRepositoryInjector());
+ engineBuilder.addDependency(TeiidServiceNames.AUTHORIZATION_VALIDATOR, AuthorizationValidator.class, engine.getAuthorizationValidatorInjector());
+ engineBuilder.addDependency(TeiidServiceNames.CACHE_RESULTSET, SessionAwareCache.class, engine.getResultSetCacheInjector());
+ engineBuilder.addDependency(TeiidServiceNames.CACHE_PREPAREDPLAN, SessionAwareCache.class, engine.getPreparedPlanCacheInjector());
+ engineBuilder.addDependency(DependencyType.OPTIONAL, TeiidServiceNames.OBJECT_REPLICATOR, ObjectReplicator.class, engine.getObjectReplicatorInjector());
+
+ engineBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
+ ServiceController<DQPCore> controller = engineBuilder.install();
+ newControllers.add(controller);
+ ServiceContainer container = controller.getServiceContainer();
+ container.addTerminateListener(shutdownListener);
+
// Register VDB deployer
context.addStep(new AbstractDeploymentChainStep() {
@Override
@@ -267,28 +327,7 @@
return services.iterator().next();
}
- private static String BS(String name) {
- return Configuration.BUFFER_SERVICE+DASH+name;
- }
- static void describeBufferManager(ModelNode node, String type, ResourceBundle bundle) {
- addAttribute(node, BS(Configuration.USE_DISK), type, bundle.getString(Configuration.USE_DISK+DESC), ModelType.BOOLEAN, false, "true"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.PROCESSOR_BATCH_SIZE), type, bundle.getString(Configuration.PROCESSOR_BATCH_SIZE+DESC), ModelType.INT, false, "512"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.CONNECTOR_BATCH_SIZE), type, bundle.getString(Configuration.CONNECTOR_BATCH_SIZE+DESC), ModelType.INT, false, "1024"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.MAX_PROCESSING_KB), type, bundle.getString(Configuration.MAX_PROCESSING_KB+DESC), ModelType.INT, false, "-1"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.MAX_RESERVED_KB), type, bundle.getString(Configuration.MAX_RESERVED_KB+DESC), ModelType.INT, false, "-1"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.MAX_FILE_SIZE), type, bundle.getString(Configuration.MAX_FILE_SIZE+DESC), ModelType.LONG, false, "2048"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.MAX_BUFFER_SPACE), type, bundle.getString(Configuration.MAX_BUFFER_SPACE+DESC), ModelType.LONG, false, "51200"); //$NON-NLS-1$
- addAttribute(node, BS(Configuration.MAX_OPEN_FILES), type, bundle.getString(Configuration.MAX_OPEN_FILES+DESC), ModelType.INT, false, "64"); //$NON-NLS-1$
- }
- private static String ORC(String name) {
- return Configuration.OBJECT_REPLICATOR+DASH+name;
- }
- static void describeObjectReplicator(ModelNode node, String type, ResourceBundle bundle) {
- addAttribute(node, ORC(Configuration.STACK), type, bundle.getString(Configuration.STACK+DESC), ModelType.STRING, false, null);
- addAttribute(node, ORC(Configuration.CLUSTER_NAME), type, bundle.getString(Configuration.CLUSTER_NAME+DESC), ModelType.STRING, false, null);
- }
-
private BufferServiceImpl buildBufferManager(ModelNode node) {
BufferServiceImpl bufferManger = new BufferServiceImpl();
@@ -296,126 +335,35 @@
return bufferManger;
}
- if (node.hasDefined(BS(Configuration.USE_DISK))) {
- bufferManger.setUseDisk(node.get(BS(Configuration.USE_DISK)).asBoolean());
+ if (Element.USE_DISK_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setUseDisk(Element.USE_DISK_ATTRIBUTE.asBoolean(node));
}
- if (node.hasDefined(BS(Configuration.PROCESSOR_BATCH_SIZE))) {
- bufferManger.setProcessorBatchSize(node.get(BS(Configuration.PROCESSOR_BATCH_SIZE)).asInt());
+ if (Element.PROCESSOR_BATCH_SIZE_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setProcessorBatchSize(Element.PROCESSOR_BATCH_SIZE_ATTRIBUTE.asInt(node));
}
- if (node.hasDefined(BS(Configuration.CONNECTOR_BATCH_SIZE))) {
- bufferManger.setConnectorBatchSize(node.get(BS(Configuration.CONNECTOR_BATCH_SIZE)).asInt());
+ if (Element.CONNECTOR_BATCH_SIZE_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setConnectorBatchSize(Element.CONNECTOR_BATCH_SIZE_ATTRIBUTE.asInt(node));
}
- if (node.hasDefined(BS(Configuration.MAX_PROCESSING_KB))) {
- bufferManger.setMaxProcessingKb(node.get(BS(Configuration.MAX_PROCESSING_KB)).asInt());
+ if (Element.MAX_PROCESSING_KB_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setMaxProcessingKb(Element.MAX_PROCESSING_KB_ATTRIBUTE.asInt(node));
}
- if (node.hasDefined(BS(Configuration.MAX_RESERVED_KB))) {
- bufferManger.setMaxReserveKb(node.get(BS(Configuration.MAX_RESERVED_KB)).asInt());
+ if (Element.MAX_RESERVED_KB_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setMaxReserveKb(Element.MAX_RESERVED_KB_ATTRIBUTE.asInt(node));
}
- if (node.hasDefined(BS(Configuration.MAX_FILE_SIZE))) {
- bufferManger.setMaxFileSize(node.get(BS(Configuration.MAX_FILE_SIZE)).asLong());
+ if (Element.MAX_FILE_SIZE_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setMaxFileSize(Element.MAX_FILE_SIZE_ATTRIBUTE.asLong(node));
}
- if (node.hasDefined(BS(Configuration.MAX_BUFFER_SPACE))) {
- bufferManger.setMaxBufferSpace(node.get(BS(Configuration.MAX_BUFFER_SPACE)).asLong());
+ if (Element.MAX_BUFFER_SPACE_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setMaxBufferSpace(Element.MAX_BUFFER_SPACE_ATTRIBUTE.asLong(node));
}
- if (node.hasDefined(BS(Configuration.MAX_OPEN_FILES))) {
- bufferManger.setMaxOpenFiles(node.get(BS(Configuration.MAX_OPEN_FILES)).asInt());
+ if (Element.MAX_OPEN_FILES_ATTRIBUTE.isDefined(node)) {
+ bufferManger.setMaxOpenFiles(Element.MAX_OPEN_FILES_ATTRIBUTE.asInt(node));
}
return bufferManger;
}
-
- private static void populateBufferManager(ModelNode operation, ModelNode model) {
-
- if (operation.hasDefined(BS(Configuration.USE_DISK))) {
- model.get(BS(Configuration.USE_DISK)).set(operation.get(BS(Configuration.USE_DISK)).asString());
- }
- if (operation.hasDefined(BS(Configuration.PROCESSOR_BATCH_SIZE))) {
- model.get(BS(Configuration.PROCESSOR_BATCH_SIZE)).set(operation.get(BS(Configuration.PROCESSOR_BATCH_SIZE)).asString());
- }
- if (operation.hasDefined(BS(Configuration.CONNECTOR_BATCH_SIZE))) {
- model.get(BS(Configuration.CONNECTOR_BATCH_SIZE)).set(operation.get(BS(Configuration.CONNECTOR_BATCH_SIZE)).asString());
- }
- if (operation.hasDefined(BS(Configuration.MAX_PROCESSING_KB))) {
- model.get(BS(Configuration.MAX_PROCESSING_KB)).set(operation.get(BS(Configuration.MAX_PROCESSING_KB)).asString());
- }
- if (operation.hasDefined(BS(Configuration.MAX_RESERVED_KB))) {
- model.get(BS(Configuration.MAX_RESERVED_KB)).set(operation.get(BS(Configuration.MAX_RESERVED_KB)).asString());
- }
- if (operation.hasDefined(BS(Configuration.MAX_FILE_SIZE))) {
- model.get(BS(Configuration.MAX_FILE_SIZE)).set(operation.get(BS(Configuration.MAX_FILE_SIZE)).asString());
- }
- if (operation.hasDefined(BS(Configuration.MAX_BUFFER_SPACE))) {
- model.get(BS(Configuration.MAX_BUFFER_SPACE)).set(operation.get(BS(Configuration.MAX_BUFFER_SPACE)).asString());
- }
- if (operation.hasDefined(BS(Configuration.MAX_BUFFER_SPACE))) {
- model.get(BS(Configuration.MAX_BUFFER_SPACE)).set(operation.get(BS(Configuration.MAX_BUFFER_SPACE)).asString());
- }
- if (operation.hasDefined(BS(Configuration.MAX_OPEN_FILES))) {
- model.get(BS(Configuration.MAX_OPEN_FILES)).set(operation.get(BS(Configuration.MAX_OPEN_FILES)).asString());
- }
- }
-
- private static void populateResultsetCache(ModelNode operation, ModelNode model) {
- if (operation.hasDefined(RSC(Configuration.NAME))) {
- model.get(RSC(Configuration.NAME)).set(operation.get(RSC(Configuration.NAME)).asString());
- }
-
- if (operation.hasDefined(RSC(Configuration.CONTAINER_NAME))) {
- model.get(RSC(Configuration.CONTAINER_NAME)).set(operation.get(RSC(Configuration.CONTAINER_NAME)).asString());
- }
+ private SessionAwareCache<CachedResults> buildResultsetCache(ModelNode node, BufferManager bufferManager) {
- if (operation.hasDefined(RSC(Configuration.ENABLE))) {
- model.get(RSC(Configuration.ENABLE)).set(operation.get(RSC(Configuration.ENABLE)).asBoolean());
- }
-
- if (operation.hasDefined(RSC(Configuration.MAX_STALENESS))) {
- model.get(RSC(Configuration.MAX_STALENESS)).set(operation.get(RSC(Configuration.MAX_STALENESS)).asInt());
- }
- }
-
- private static void populateObjectReplicator(ModelNode operation, ModelNode model) {
- if (operation.hasDefined(ORC(Configuration.STACK))) {
- model.get(ORC(Configuration.STACK)).set(operation.get(ORC(Configuration.STACK)).asString());
- }
-
- if (operation.hasDefined(ORC(Configuration.CLUSTER_NAME))) {
- model.get(ORC(Configuration.CLUSTER_NAME)).set(operation.get(ORC(Configuration.CLUSTER_NAME)).asString());
- }
- }
-
- private static void populatePreparedPlanCache(ModelNode operation, ModelNode model) {
- if (operation.hasDefined(PPC(Configuration.MAX_ENTRIES))) {
- model.get(PPC(Configuration.MAX_ENTRIES)).set(operation.get(PPC(Configuration.MAX_ENTRIES)).asInt());
- }
- if (operation.hasDefined(PPC(Configuration.MAX_AGE_IN_SECS))) {
- model.get(PPC(Configuration.MAX_AGE_IN_SECS)).set(operation.get(PPC(Configuration.MAX_AGE_IN_SECS)).asInt());
- }
- if (operation.hasDefined(PPC(Configuration.MAX_STALENESS))) {
- model.get(PPC(Configuration.MAX_STALENESS)).set(operation.get(PPC(Configuration.MAX_STALENESS)).asInt());
- }
- }
-
- private static String RSC(String name) {
- return Configuration.RESULTSET_CACHE+DASH+name;
- }
- private static void describeResultsetCache(ModelNode node, String type, ResourceBundle bundle) {
- addAttribute(node, RSC(Configuration.NAME), type, bundle.getString(RSC(Configuration.NAME)+DESC), ModelType.STRING, false, "resultset"); //$NON-NLS-1$
- addAttribute(node, RSC(Configuration.MAX_STALENESS), type, bundle.getString(Configuration.MAX_STALENESS+DESC), ModelType.INT, false, "60");//$NON-NLS-1$
- addAttribute(node, RSC(Configuration.ENABLE), type, bundle.getString(Configuration.ENABLE+DESC), ModelType.BOOLEAN, false, null);
- addAttribute(node, RSC(Configuration.CONTAINER_NAME), type, bundle.getString(Configuration.CONTAINER_NAME+DESC), ModelType.STRING, false, null);
- }
-
- private static String PPC(String name) {
- return Configuration.PREPAREDPLAN_CACHE+DASH+name;
- }
- private static void describePreparedPlanCache(ModelNode node, String type, ResourceBundle bundle) {
- addAttribute(node, PPC(Configuration.MAX_ENTRIES), type, bundle.getString(Configuration.MAX_ENTRIES+DESC), ModelType.INT, false, "512"); //$NON-NLS-1$
- addAttribute(node, PPC(Configuration.MAX_AGE_IN_SECS), type, bundle.getString(Configuration.MAX_AGE_IN_SECS+DESC), ModelType.INT, false, "28800");//$NON-NLS-1$
- addAttribute(node, PPC(Configuration.MAX_STALENESS), type, bundle.getString(Configuration.MAX_STALENESS+DESC), ModelType.INT, false, "0");//$NON-NLS-1$
- }
-
- private SessionAwareCache<CachedResults> buildResultsetCache(ModelNode operation, BufferManager bufferManager) {
-
CacheConfiguration cacheConfig = new CacheConfiguration();
// these settings are not really used; they are defined by infinispan
cacheConfig.setMaxEntries(1024);
@@ -424,17 +372,17 @@
cacheConfig.setLocation("resultset"); //$NON-NLS-1$
cacheConfig.setMaxStaleness(60);
- if (operation.hasDefined(RSC(Configuration.ENABLE))) {
- if (!operation.get(RSC(Configuration.ENABLE)).asBoolean()) {
+ if (Element.RSC_ENABLE_ATTRIBUTE.isDefined(node)) {
+ if (!Element.RSC_ENABLE_ATTRIBUTE.asBoolean(node)) {
return null;
}
}
ClusterableCacheFactory cacheFactory = null;
- if (operation.hasDefined(RSC(Configuration.CONTAINER_NAME))) {
+ if (Element.RSC_CONTAINER_NAME_ELEMENT.isDefined(node)) {
cacheFactory = new ClusterableCacheFactory();
- cacheFactory.setCacheManager(operation.get(RSC(Configuration.CONTAINER_NAME)).asString());
+ cacheFactory.setCacheManager(Element.RSC_CONTAINER_NAME_ELEMENT.asString(node));
}
else {
SessionAwareCache<CachedResults> resultsetCache = new SessionAwareCache<CachedResults>(new DefaultCacheFactory(), SessionAwareCache.Type.RESULTSET, cacheConfig);
@@ -442,15 +390,15 @@
return resultsetCache;
}
- if (operation.hasDefined(RSC(Configuration.NAME))) {
- cacheFactory.setResultsetCacheName(operation.get(RSC(Configuration.NAME)).asString());
+ if (Element.RSC_NAME_ELEMENT.isDefined(node)) {
+ cacheFactory.setResultsetCacheName(Element.RSC_NAME_ELEMENT.asString(node));
}
else {
cacheFactory.setResultsetCacheName("resultset"); //$NON-NLS-1$
}
- if (operation.hasDefined(RSC(Configuration.MAX_STALENESS))) {
- cacheConfig.setMaxStaleness(operation.get(RSC(Configuration.MAX_STALENESS)).asInt());
+ if (Element.RSC_MAX_STALENESS_ELEMENT.isDefined(node)) {
+ cacheConfig.setMaxStaleness(Element.RSC_MAX_STALENESS_ELEMENT.asInt(node));
}
SessionAwareCache<CachedResults> resultsetCache = new SessionAwareCache<CachedResults>(cacheFactory, SessionAwareCache.Type.RESULTSET, cacheConfig);
@@ -461,22 +409,22 @@
private SessionAwareCache<PreparedPlan> buildPreparedPlanCache(ModelNode node, BufferManager bufferManager) {
CacheConfiguration cacheConfig = new CacheConfiguration();
- if (node.hasDefined(PPC(Configuration.MAX_ENTRIES))) {
- cacheConfig.setMaxEntries(node.get(PPC(Configuration.MAX_ENTRIES)).asInt());
+ if (Element.PPC_MAX_ENTRIES_ATTRIBUTE.isDefined(node)) {
+ cacheConfig.setMaxEntries(Element.PPC_MAX_ENTRIES_ATTRIBUTE.asInt(node));
}
else {
cacheConfig.setMaxEntries(512);
}
- if (node.hasDefined(PPC(Configuration.MAX_AGE_IN_SECS))) {
- cacheConfig.setMaxAgeInSeconds(node.get(PPC(Configuration.MAX_AGE_IN_SECS)).asInt());
+ if (Element.PPC_MAX_AGE_IN_SECS_ATTRIBUTE.isDefined(node)) {
+ cacheConfig.setMaxAgeInSeconds(Element.PPC_MAX_AGE_IN_SECS_ATTRIBUTE.asInt(node));
}
else {
cacheConfig.setMaxAgeInSeconds(28800);
}
- if (node.hasDefined(PPC(Configuration.MAX_STALENESS))) {
- cacheConfig.setMaxStaleness(node.get(PPC(Configuration.MAX_STALENESS)).asInt());
+ if (Element.PPC_MAX_STALENESS_ATTRIBUTE.isDefined(node)) {
+ cacheConfig.setMaxStaleness(Element.PPC_MAX_STALENESS_ATTRIBUTE.asInt(node));
}
else {
cacheConfig.setMaxStaleness(0);
@@ -490,4 +438,40 @@
return cache;
}
+
+ private RuntimeEngineDeployer buildQueryEngine(ModelNode node) {
+ RuntimeEngineDeployer engine = new RuntimeEngineDeployer();
+
+ if (Element.MAX_THREADS_ELEMENT.isDefined(node)) {
+ engine.setMaxThreads(Element.MAX_THREADS_ELEMENT.asInt(node));
+ }
+ if (Element.MAX_ACTIVE_PLANS_ELEMENT.isDefined(node)) {
+ engine.setMaxActivePlans(Element.MAX_ACTIVE_PLANS_ELEMENT.asInt(node));
+ }
+ if (Element.USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT.isDefined(node)) {
+ engine.setUserRequestSourceConcurrency(Element.USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT.asInt(node));
+ }
+ if (Element.TIME_SLICE_IN_MILLI_ELEMENT.isDefined(node)) {
+ engine.setTimeSliceInMilli(Element.TIME_SLICE_IN_MILLI_ELEMENT.asInt(node));
+ }
+ if (Element.MAX_ROWS_FETCH_SIZE_ELEMENT.isDefined(node)) {
+ engine.setMaxRowsFetchSize(Element.MAX_ROWS_FETCH_SIZE_ELEMENT.asInt(node));
+ }
+ if (Element.LOB_CHUNK_SIZE_IN_KB_ELEMENT.isDefined(node)) {
+ engine.setLobChunkSizeInKB(Element.LOB_CHUNK_SIZE_IN_KB_ELEMENT.asInt(node));
+ }
+ if (Element.QUERY_THRESHOLD_IN_SECS_ELEMENT.isDefined(node)) {
+ engine.setQueryThresholdInSecs(Element.QUERY_THRESHOLD_IN_SECS_ELEMENT.asInt(node));
+ }
+ if (Element.MAX_SOURCE_ROWS_ELEMENT.isDefined(node)) {
+ engine.setMaxSourceRows(Element.MAX_SOURCE_ROWS_ELEMENT.asInt(node));
+ }
+ if (Element.EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT.isDefined(node)) {
+ engine.setExceptionOnMaxSourceRows(Element.EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT.asBoolean(node));
+ }
+ if (Element.DETECTING_CHANGE_EVENTS_ELEMENT.isDefined(node)) {
+ engine.setDetectingChangeEvents(Element.DETECTING_CHANGE_EVENTS_ELEMENT.asBoolean(node));
+ }
+ return engine;
+ }
}
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidExtension.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidExtension.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidExtension.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -21,8 +21,13 @@
*/
package org.teiid.jboss;
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-import static org.teiid.jboss.Configuration.addAttribute;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.ADD;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.ATTRIBUTES;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIBE;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.HEAD_COMMENT_ALLOWED;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REMOVE;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.TAIL_COMMENT_ALLOWED;
import java.util.Locale;
import java.util.ResourceBundle;
@@ -36,7 +41,6 @@
import org.jboss.as.controller.registry.AttributeAccess.Storage;
import org.jboss.as.controller.registry.ManagementResourceRegistration;
import org.jboss.dmr.ModelNode;
-import org.jboss.dmr.ModelType;
import org.teiid.logging.Log4jListener;
import org.teiid.logging.LogManager;
@@ -47,8 +51,8 @@
public static final String TEIID_SUBSYSTEM = "teiid"; //$NON-NLS-1$
private static TeiidSubsystemParser parser = new TeiidSubsystemParser();
- private static QueryEngineAdd ENGINE_ADD = new QueryEngineAdd();
- private static QueryEngineRemove ENGINE_REMOVE = new QueryEngineRemove();
+ private static TransportAdd TRANSPORT_ADD = new TransportAdd();
+ private static TransportRemove TRANSPORT_REMOVE = new TransportRemove();
private static TranslatorAdd TRANSLATOR_ADD = new TranslatorAdd();
private static TranslatorRemove TRANSLATOR_REMOVE = new TranslatorRemove();
private static TeiidBootServicesAdd TEIID_BOOT_ADD = new TeiidBootServicesAdd();
@@ -69,16 +73,16 @@
teiidSubsystem.registerOperationHandler(DESCRIBE, TEIID_DESCRIBE, TEIID_DESCRIBE, false);
// Translator Subsystem
- final ManagementResourceRegistration translatorSubsystem = teiidSubsystem.registerSubModel(PathElement.pathElement(Configuration.TRANSLATOR), new DescriptionProvider() {
+ final ManagementResourceRegistration translatorSubsystem = teiidSubsystem.registerSubModel(PathElement.pathElement(Element.TRANSLATOR_ELEMENT.getLocalName()), new DescriptionProvider() {
@Override
public ModelNode getModelDescription(Locale locale) {
final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
final ModelNode node = new ModelNode();
- node.get(DESCRIPTION).set(bundle.getString(Configuration.TRANSLATOR+Configuration.DESC));
+ node.get(DESCRIPTION).set(Element.TRANSLATOR_ELEMENT.getDescription(bundle));
node.get(HEAD_COMMENT_ALLOWED).set(true);
node.get(TAIL_COMMENT_ALLOWED).set(true);
- addAttribute(node, Configuration.TRANSLATOR_MODULE, ATTRIBUTES, bundle.getString(Configuration.TRANSLATOR_MODULE+Configuration.DESC), ModelType.STRING, true, null);
+ Element.TRANSLATOR_MODULE_ATTRIBUTE.describe(node, ATTRIBUTES, bundle);
return node;
}
});
@@ -87,25 +91,27 @@
// Query engine subsystem
- final ManagementResourceRegistration engineSubsystem = teiidSubsystem.registerSubModel(PathElement.pathElement(Configuration.QUERY_ENGINE), new DescriptionProvider() {
+ final ManagementResourceRegistration transportModel = teiidSubsystem.registerSubModel(PathElement.pathElement(Element.TRANSPORT_ELEMENT.getLocalName()), new DescriptionProvider() {
@Override
public ModelNode getModelDescription(Locale locale) {
final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
final ModelNode node = new ModelNode();
- node.get(DESCRIPTION).set(bundle.getString(Configuration.QUERY_ENGINE+Configuration.DESC));
+ node.get(DESCRIPTION).set(Element.TRANSPORT_ELEMENT.getDescription(bundle));
node.get(HEAD_COMMENT_ALLOWED).set(true);
node.get(TAIL_COMMENT_ALLOWED).set(true);
- QueryEngineAdd.describeQueryEngine(node, ATTRIBUTES, bundle);
+
+ TransportAdd.transportDescribe(node, ATTRIBUTES, bundle);
+
return node;
}
});
- engineSubsystem.registerOperationHandler(ADD, ENGINE_ADD, ENGINE_ADD, false);
- engineSubsystem.registerOperationHandler(REMOVE, ENGINE_REMOVE, ENGINE_REMOVE, false);
+ transportModel.registerOperationHandler(ADD, TRANSPORT_ADD, TRANSPORT_ADD, false);
+ transportModel.registerOperationHandler(REMOVE, TRANSPORT_REMOVE, TRANSPORT_REMOVE, false);
- engineSubsystem.registerReadOnlyAttribute(RUNTIME_VERSION, new GetRuntimeVersion(RUNTIME_VERSION), Storage.RUNTIME);
- engineSubsystem.registerReadOnlyAttribute(ACTIVE_SESSION_COUNT, new GetActiveSessionsCount(ACTIVE_SESSION_COUNT), Storage.RUNTIME);
+ teiidSubsystem.registerReadOnlyAttribute(RUNTIME_VERSION, new GetRuntimeVersion(RUNTIME_VERSION), Storage.RUNTIME);
+ teiidSubsystem.registerReadOnlyAttribute(ACTIVE_SESSION_COUNT, new GetActiveSessionsCount(ACTIVE_SESSION_COUNT), Storage.RUNTIME);
// teiid level admin api operation handlers
new GetTranslator().register(teiidSubsystem);
@@ -124,17 +130,17 @@
new RemoveAnyAuthenticatedDataRole().register(teiidSubsystem);
// engine level admin api handlers
- new ListRequests().register(engineSubsystem);
- new ListSessions().register(engineSubsystem);
- new RequestsPerSession().register(engineSubsystem);
- new RequestsPerVDB().register(engineSubsystem);
- new GetLongRunningQueries().register(engineSubsystem);
- new TerminateSession().register(engineSubsystem);
- new CancelRequest().register(engineSubsystem);
- new WorkerPoolStatistics().register(engineSubsystem);
- new ListTransactions().register(engineSubsystem);
- new TerminateTransaction().register(engineSubsystem);
- new ExecuteQuery().register(engineSubsystem);
+ new ListRequests().register(teiidSubsystem);
+ new ListSessions().register(teiidSubsystem);
+ new RequestsPerSession().register(teiidSubsystem);
+ new RequestsPerVDB().register(teiidSubsystem);
+ new GetLongRunningQueries().register(teiidSubsystem);
+ new TerminateSession().register(teiidSubsystem);
+ new CancelRequest().register(teiidSubsystem);
+ new WorkerPoolStatistics().register(teiidSubsystem);
+ new ListTransactions().register(teiidSubsystem);
+ new TerminateTransaction().register(teiidSubsystem);
+ new ExecuteQuery().register(teiidSubsystem);
}
@Override
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidServiceNames.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidServiceNames.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidServiceNames.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -24,10 +24,11 @@
import org.jboss.msc.service.ServiceName;
public class TeiidServiceNames {
- private static ServiceName ENGINE = ServiceName.JBOSS.append("teiid", "query-engine"); //$NON-NLS-1$ //$NON-NLS-2$
+ public static ServiceName ENGINE = ServiceName.JBOSS.append("teiid", "query-engine"); //$NON-NLS-1$ //$NON-NLS-2$
public static ServiceName TRANSLATOR_REPO = ServiceName.JBOSS.append("teiid", "translator-repository");//$NON-NLS-1$ //$NON-NLS-2$
public static ServiceName VDB_REPO = ServiceName.JBOSS.append("teiid", "vdb-repository");//$NON-NLS-1$ //$NON-NLS-2$
- private static ServiceName TRANSLATOR_BASE = ServiceName.JBOSS.append("teiid", "translator");//$NON-NLS-1$ //$NON-NLS-2$
+ public static ServiceName TRANSLATOR_BASE = ServiceName.JBOSS.append("teiid", "translator");//$NON-NLS-1$ //$NON-NLS-2$
+ public static ServiceName TRANSPORT_BASE = ServiceName.JBOSS.append("teiid", "transport");//$NON-NLS-1$ //$NON-NLS-2$
public static ServiceName BUFFER_DIR = ServiceName.JBOSS.append("teiid", "buffer.dir");//$NON-NLS-1$ //$NON-NLS-2$
public static ServiceName DATA_DIR = ServiceName.JBOSS.append("teiid", "data.dir");//$NON-NLS-1$ //$NON-NLS-2$
public static ServiceName BUFFER_MGR = ServiceName.JBOSS.append("teiid", "buffer-mgr");//$NON-NLS-1$ //$NON-NLS-2$
@@ -41,13 +42,9 @@
public static ServiceName translatorServiceName(String name) {
- return TRANSLATOR_BASE.append(name);
+ return ServiceName.of(TRANSLATOR_BASE, name);
}
- public static ServiceName engineServiceName(String name) {
- return ENGINE.append(name);
- }
-
public static ServiceName vdbServiceName(String vdbName, int version) {
return VDB_SVC_BASE.append(vdbName, String.valueOf(version));
}
@@ -55,4 +52,8 @@
public static ServiceName executorServiceName(String poolName) {
return ServiceName.JBOSS.append("thread", "executor", poolName); //$NON-NLS-1$ //$NON-NLS-2$
}
+
+ public static ServiceName transportServiceName(String name) {
+ return ServiceName.of(TRANSPORT_BASE, name);
+ }
}
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemDescribe.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemDescribe.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemDescribe.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -49,12 +49,12 @@
node.get(ModelDescriptionConstants.TAIL_COMMENT_ALLOWED).set(true);
node.get(ModelDescriptionConstants.NAMESPACE).set(Namespace.CURRENT.getUri());
- TeiidBootServicesAdd.describeTeiidRoot(bundle, ATTRIBUTES, node);
- node.get(CHILDREN, Configuration.QUERY_ENGINE, DESCRIPTION).set(bundle.getString(Configuration.QUERY_ENGINE+Configuration.DESC));
- node.get(CHILDREN, Configuration.QUERY_ENGINE, REQUIRED).set(true);
+ TeiidBootServicesAdd.describeTeiid(node, ATTRIBUTES, bundle);
+ node.get(CHILDREN, Element.TRANSPORT_ELEMENT.getLocalName(), DESCRIPTION).set(Element.TRANSPORT_ELEMENT.getDescription(bundle));
+ node.get(CHILDREN, Element.TRANSPORT_ELEMENT.getLocalName(), REQUIRED).set(true);
- node.get(CHILDREN, Configuration.TRANSLATOR, DESCRIPTION).set(bundle.getString(Configuration.TRANSLATOR+Configuration.DESC));
- node.get(CHILDREN, Configuration.TRANSLATOR, REQUIRED).set(true);
+ node.get(CHILDREN, Element.TRANSLATOR_ELEMENT.getLocalName(), DESCRIPTION).set(Element.TRANSLATOR_ELEMENT.getDescription(bundle));
+ node.get(CHILDREN, Element.TRANSLATOR_ELEMENT.getLocalName(), REQUIRED).set(true);
return node;
}
@@ -72,25 +72,25 @@
TeiidBootServicesAdd.populate(subModel, subsystemAdd);
result.add(subsystemAdd);
- if (subModel.hasDefined(Configuration.QUERY_ENGINE)) {
- for (Property container : subModel.get(Configuration.QUERY_ENGINE).asPropertyList()) {
+ if (subModel.hasDefined(Element.TRANSPORT_ELEMENT.getLocalName())) {
+ for (Property container : subModel.get(Element.TRANSPORT_ELEMENT.getLocalName()).asPropertyList()) {
ModelNode address = rootAddress.toModelNode();
- address.add(Configuration.QUERY_ENGINE, container.getName());
+ address.add(Element.TRANSPORT_ELEMENT.getLocalName(), container.getName());
final ModelNode addOperation = new ModelNode();
addOperation.get(OP).set(ADD);
addOperation.get(OP_ADDR).set(address);
- QueryEngineAdd.populate(container.getValue(), addOperation);
+ TransportAdd.populate(container.getValue(), addOperation);
result.add(addOperation);
}
}
- if (subModel.hasDefined(Configuration.TRANSLATOR)) {
- for (Property container : subModel.get(Configuration.TRANSLATOR).asPropertyList()) {
+ if (subModel.hasDefined(Element.TRANSLATOR_ELEMENT.getLocalName())) {
+ for (Property container : subModel.get(Element.TRANSLATOR_ELEMENT.getLocalName()).asPropertyList()) {
ModelNode address = rootAddress.toModelNode();
- address.add(Configuration.TRANSLATOR, container.getName());
+ address.add(Element.TRANSLATOR_ELEMENT.getLocalName(), container.getName());
final ModelNode addOperation = new ModelNode();
addOperation.get(OP).set(ADD);
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemParser.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemParser.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TeiidSubsystemParser.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -54,18 +54,38 @@
}
writeElement(writer, Element.ALLOW_ENV_FUNCTION_ELEMENT, node);
- writeElement(writer, Element.ASYNC_THREAD_GROUP_ELEMENT, node);
+ writeElement(writer, Element.ASYNC_THREAD_POOL_ELEMENT, node);
if (like(node, Element.BUFFER_SERVICE_ELEMENT)){
writer.writeStartElement(Element.BUFFER_SERVICE_ELEMENT.getLocalName());
writeBufferService(writer, node);
writer.writeEndElement();
}
+
+ writeElement(writer, Element.MAX_THREADS_ELEMENT, node);
+ writeElement(writer, Element.MAX_ACTIVE_PLANS_ELEMENT, node);
+ writeElement(writer, Element.USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT, node);
+ writeElement(writer, Element.TIME_SLICE_IN_MILLI_ELEMENT, node);
+ writeElement(writer, Element.MAX_ROWS_FETCH_SIZE_ELEMENT, node);
+ writeElement(writer, Element.LOB_CHUNK_SIZE_IN_KB_ELEMENT, node);
+ writeElement(writer, Element.QUERY_THRESHOLD_IN_SECS_ELEMENT, node);
+ writeElement(writer, Element.MAX_SOURCE_ROWS_ELEMENT, node);
+ writeElement(writer, Element.EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT, node);
+ writeElement(writer, Element.DETECTING_CHANGE_EVENTS_ELEMENT, node);
+
- writeElement(writer, Element.AUTHORIZATION_VALIDATOR_MODULE_ELEMENT, node);
- writeElement(writer, Element.POLICY_DECIDER_MODULE_ELEMENT, node);
+ if (like(node, Element.AUTHORIZATION_VALIDATOR_ELEMENT)) {
+ writer.writeStartElement(Element.AUTHORIZATION_VALIDATOR_ELEMENT.getLocalName());
+ writer.writeAttribute(Element.AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE.getLocalName(), node.get(Element.AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE.getModelName()).asString());
+ writer.writeEndElement();
+ }
-
+ if (like(node, Element.POLICY_DECIDER_ELEMENT)) {
+ writer.writeStartElement(Element.POLICY_DECIDER_ELEMENT.getLocalName());
+ writer.writeAttribute(Element.POLICY_DECIDER_MODULE_ATTRIBUTE.getLocalName(), node.get(Element.POLICY_DECIDER_MODULE_ATTRIBUTE.getModelName()).asString());
+ writer.writeEndElement();
+ }
+
if (like(node, Element.RESULTSET_CACHE_ELEMENT)){
writer.writeStartElement(Element.RESULTSET_CACHE_ELEMENT.getLocalName());
writeResultsetCacheConfiguration(writer, node);
@@ -83,18 +103,18 @@
writeObjectReplicatorConfiguration(writer, node);
writer.writeEndElement();
}
-
- if (has(node, Element.QUERY_ENGINE_ELEMENT.getLocalName())) {
- ArrayList<String> engines = new ArrayList<String>(node.get(Element.QUERY_ENGINE_ELEMENT.getLocalName()).keys());
- Collections.sort(engines);
- if (!engines.isEmpty()) {
- for (String engine:engines) {
- writer.writeStartElement(Element.QUERY_ENGINE_ELEMENT.getLocalName());
- writeQueryEngine(writer, node.get(Element.QUERY_ENGINE_ELEMENT.getLocalName(), engine), engine);
+
+ if (has(node, Element.TRANSPORT_ELEMENT.getLocalName())) {
+ ArrayList<String> transports = new ArrayList<String>(node.get(Element.TRANSPORT_ELEMENT.getLocalName()).keys());
+ Collections.sort(transports);
+ if (!transports.isEmpty()) {
+ for (String transport:transports) {
+ writer.writeStartElement(Element.TRANSPORT_ELEMENT.getLocalName());
+ writeTransportConfiguration(writer, node.get(Element.TRANSPORT_ELEMENT.getLocalName(), transport), transport);
writer.writeEndElement();
}
- }
- }
+ }
+ }
if (has(node, Element.TRANSLATOR_ELEMENT.getLocalName())) {
ArrayList<String> translators = new ArrayList<String>(node.get(Element.TRANSLATOR_ELEMENT.getLocalName()).keys());
@@ -121,109 +141,66 @@
}
// write the elements according to the schema defined.
- private void writeQueryEngine( XMLExtendedStreamWriter writer, ModelNode node, String engineName) throws XMLStreamException {
- writer.writeAttribute(Element.ENGINE_NAME_ATTRIBUTE.getLocalName(), engineName);
+ private void writeTransportConfiguration( XMLExtendedStreamWriter writer, ModelNode node, String transportName) throws XMLStreamException {
- writeElement(writer, Element.MAX_THREADS_ELEMENT, node);
- writeElement(writer, Element.MAX_ACTIVE_PLANS_ELEMENT, node);
- writeElement(writer, Element.USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT, node);
- writeElement(writer, Element.TIME_SLICE_IN_MILLI_ELEMENT, node);
- writeElement(writer, Element.MAX_ROWS_FETCH_SIZE_ELEMENT, node);
- writeElement(writer, Element.LOB_CHUNK_SIZE_IN_KB_ELEMENT, node);
- writeElement(writer, Element.AUTHORIZATION_VALIDATOR_MODULE_ELEMENT, node);
- writeElement(writer, Element.POLICY_DECIDER_MODULE_ELEMENT, node);
- writeElement(writer, Element.QUERY_THRESHOLD_IN_SECS_ELEMENT, node);
- writeElement(writer, Element.MAX_SOURCE_ROWS_ELEMENT, node);
- writeElement(writer, Element.EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT, node);
- writeElement(writer, Element.MAX_ODBC_LOB_SIZE_ALLOWED_ELEMENT, node);
- writeElement(writer, Element.OBJECT_REPLICATOR_ELEMENT, node);
- writeElement(writer, Element.DETECTING_CHANGE_EVENTS_ELEMENT, node);
+ writer.writeAttribute(Element.TRANSPORT_NAME_ATTRIBUTE.getLocalName(), transportName);
+ writeAttribute(writer, Element.TRANSPORT_SOCKET_BINDING_ATTRIBUTE, node);
+ writeAttribute(writer, Element.TRANSPORT_PROTOCOL_ATTRIBUTE, node);
+ writeAttribute(writer, Element.TRANSPORT_MAX_SOCKET_THREADS_ATTRIBUTE, node);
+ writeAttribute(writer, Element.TRANSPORT_IN_BUFFER_SIZE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.TRANSPORT_OUT_BUFFER_SIZE_ATTRIBUTE, node);
- if (node.hasDefined(Element.SECURITY_DOMAIN_ELEMENT.getLocalName())) {
- List<ModelNode> domains = node.get(Element.SECURITY_DOMAIN_ELEMENT.getLocalName()).asList();
- writeElement(writer, Element.SECURITY_DOMAIN_ELEMENT, domains);
+ // authentication
+ if (like(node, Element.AUTHENTICATION_ELEMENT)) {
+ writer.writeStartElement(Element.AUTHENTICATION_ELEMENT.getLocalName());
+ writeAttribute(writer, Element.AUTHENTICATION_SECURITY_DOMAIN_ATTRIBUTE, node);
+ writeAttribute(writer, Element.AUTHENTICATION_MAX_SESSIONS_ALLOWED_ATTRIBUTE, node);
+ writeAttribute(writer, Element.AUTHENTICATION_SESSION_EXPIRATION_TIME_LIMIT_ATTRIBUTE, node);
+ writer.writeEndElement();
}
- writeElement(writer, Element.MAX_SESSIONS_ALLOWED_ELEMENT, node);
- writeElement(writer, Element.SESSION_EXPIRATION_TIME_LIMIT_ELEMENT, node);
-
-
- //jdbc
- if (like(node, Element.JDBC_ELEMENT)) {
- writer.writeStartElement(Element.JDBC_ELEMENT.getLocalName());
- writeJDBCSocketConfiguration(writer, node);
+ if (like(node, Element.PG_ELEMENT)) {
+ writer.writeStartElement(Element.PG_ELEMENT.getLocalName());
+ writeAttribute(writer, Element.PG_MAX_LOB_SIZE_ALLOWED_ELEMENT, node);
+ writeAttribute(writer, Element.PG_AUTHENTICATION_TYPE_ATTRIBUTE, node);
writer.writeEndElement();
- }
+ }
- //odbc
- if (like(node, Element.ODBC_ELEMENT)) {
- writer.writeStartElement(Element.ODBC_ELEMENT.getLocalName());
- writeODBCSocketConfiguration(writer, node);
+ if (like(node, Element.SSL_ELEMENT)) {
+ writer.writeStartElement(Element.SSL_ELEMENT.getLocalName());
+
+ writeAttribute(writer, Element.SSL_MODE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.SSL_AUTH_MODE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.SSL_SSL_PROTOCOL_ATTRIBUTE, node);
+ writeAttribute(writer, Element.SSL_KEY_MANAGEMENT_ALG_ATTRIBUTE, node);
+
+ if (like(node, Element.SSL_KETSTORE_ELEMENT)) {
+ writer.writeStartElement(Element.SSL_KETSTORE_ELEMENT.getLocalName());
+ writeAttribute(writer, Element.SSL_KETSTORE_NAME_ATTRIBUTE, node);
+ writeAttribute(writer, Element.SSL_KETSTORE_PASSWORD_ATTRIBUTE, node);
+ writeAttribute(writer, Element.SSL_KETSTORE_TYPE_ATTRIBUTE, node);
+ writer.writeEndElement();
+ }
+
+ if (like(node, Element.SSL_TRUSTSTORE_ELEMENT)) {
+ writer.writeStartElement(Element.SSL_TRUSTSTORE_ELEMENT.getLocalName());
+ writeAttribute(writer, Element.SSL_TRUSTSTORE_NAME_ATTRIBUTE, node);
+ writeAttribute(writer, Element.SSL_TRUSTSTORE_PASSWORD_ATTRIBUTE, node);
+ writer.writeEndElement();
+ }
writer.writeEndElement();
}
}
- private void writeJDBCSocketConfiguration(XMLExtendedStreamWriter writer, ModelNode node) throws XMLStreamException {
- writeAttribute(writer, Element.JDBC_SOCKET_BINDING_ATTRIBUTE, node);
- writeAttribute(writer, Element.JDBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE, node);
- writeAttribute(writer, Element.JDBC_IN_BUFFER_SIZE_ATTRIBUTE, node);
- writeAttribute(writer, Element.JDBC_OUT_BUFFER_SIZE_ATTRIBUTE, node);
-
- // SSL
- if (like(node, Element.JDBC_SSL_ELEMENT)) {
- writer.writeStartElement(Element.JDBC_SSL_ELEMENT.getLocalName());
- writeJDBCSSLConfiguration(writer, node);
- writer.writeEndElement();
- }
- }
-
- private void writeJDBCSSLConfiguration(XMLExtendedStreamWriter writer, ModelNode node) throws XMLStreamException {
- writeElement(writer, Element.JDBC_SSL_MODE_ELEMENT, node);
- writeElement(writer, Element.JDBC_AUTH_MODE_ELEMENT, node);
- writeElement(writer, Element.JDBC_SSL_PROTOCOL_ELEMENT, node);
- writeElement(writer, Element.JDBC_KEY_MANAGEMENT_ALG_ELEMENT, node);
- writeElement(writer, Element.JDBC_KEY_STORE_FILE_ELEMENT, node);
- writeElement(writer, Element.JDBC_KEY_STORE_PASSWD_ELEMENT, node);
- writeElement(writer, Element.JDBC_KEY_STORE_TYPE_ELEMENT, node);
- writeElement(writer, Element.JDBC_TRUST_FILE_ELEMENT, node);
- writeElement(writer, Element.JDBC_TRUST_PASSWD_ELEMENT, node);
- }
-
- private void writeODBCSocketConfiguration(XMLExtendedStreamWriter writer, ModelNode node) throws XMLStreamException {
- writeAttribute(writer, Element.ODBC_SOCKET_BINDING_ATTRIBUTE, node);
- writeAttribute(writer, Element.ODBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE, node);
- writeAttribute(writer, Element.ODBC_IN_BUFFER_SIZE_ATTRIBUTE, node);
- writeAttribute(writer, Element.ODBC_OUT_BUFFER_SIZE_ATTRIBUTE, node);
-
- // SSL
- if (like(node, Element.ODBC_SSL_ELEMENT)) {
- writer.writeStartElement(Element.ODBC_SSL_ELEMENT.getLocalName());
- writeODBCSSLConfiguration(writer, node);
- writer.writeEndElement();
- }
- }
-
- private void writeODBCSSLConfiguration(XMLExtendedStreamWriter writer, ModelNode node) throws XMLStreamException {
- writeElement(writer, Element.ODBC_SSL_MODE_ELEMENT, node);
- writeElement(writer, Element.ODBC_AUTH_MODE_ELEMENT, node);
- writeElement(writer, Element.ODBC_SSL_PROTOCOL_ELEMENT, node);
- writeElement(writer, Element.ODBC_KEY_MANAGEMENT_ALG_ELEMENT, node);
- writeElement(writer, Element.ODBC_KEY_STORE_FILE_ELEMENT, node);
- writeElement(writer, Element.ODBC_KEY_STORE_PASSWD_ELEMENT, node);
- writeElement(writer, Element.ODBC_KEY_STORE_TYPE_ELEMENT, node);
- writeElement(writer, Element.ODBC_TRUST_FILE_ELEMENT, node);
- writeElement(writer, Element.ODBC_TRUST_PASSWD_ELEMENT, node);
- }
-
private void writeBufferService(XMLExtendedStreamWriter writer, ModelNode node) throws XMLStreamException {
- writeElement(writer, Element.USE_DISK_ELEMENT, node);
- writeElement(writer, Element.PROCESSOR_BATCH_SIZE_ELEMENT, node);
- writeElement(writer, Element.CONNECTOR_BATCH_SIZE_ELEMENT, node);
- writeElement(writer, Element.MAX_PROCESSING_KB_ELEMENT, node);
- writeElement(writer, Element.MAX_RESERVED_KB_ELEMENT, node);
- writeElement(writer, Element.MAX_FILE_SIZE_ELEMENT, node);
- writeElement(writer, Element.MAX_BUFFER_SPACE_ELEMENT, node);
- writeElement(writer, Element.MAX_OPEN_FILES_ELEMENT, node);
+ writeAttribute(writer, Element.USE_DISK_ATTRIBUTE, node);
+ writeAttribute(writer, Element.PROCESSOR_BATCH_SIZE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.CONNECTOR_BATCH_SIZE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.MAX_PROCESSING_KB_ATTRIBUTE, node);
+ writeAttribute(writer, Element.MAX_RESERVED_KB_ATTRIBUTE, node);
+ writeAttribute(writer, Element.MAX_FILE_SIZE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.MAX_BUFFER_SPACE_ATTRIBUTE, node);
+ writeAttribute(writer, Element.MAX_OPEN_FILES_ATTRIBUTE, node);
}
private void writeResultsetCacheConfiguration(XMLExtendedStreamWriter writer, ModelNode node) throws XMLStreamException {
@@ -247,7 +224,7 @@
if (node.isDefined()) {
Set<String> keys = node.keys();
for (String key:keys) {
- if (key.startsWith(element.getModelName())) {
+ if (key.startsWith(element.getLocalName())) {
return true;
}
}
@@ -257,23 +234,21 @@
private void writeElement(final XMLExtendedStreamWriter writer, final Element element, final ModelNode node) throws XMLStreamException {
if (has(node, element.getModelName())) {
- writer.writeStartElement(element.getLocalName());
- writer.writeCharacters(node.get(element.getModelName()).asString());
- writer.writeEndElement();
+ String value = node.get(element.getModelName()).asString();
+ if (!element.sameAsDefault(value)) {
+ writer.writeStartElement(element.getLocalName());
+ writer.writeCharacters(value);
+ writer.writeEndElement();
+ }
}
}
- private void writeElement(final XMLExtendedStreamWriter writer, final Element element, final List<ModelNode> nodes) throws XMLStreamException {
- for (ModelNode node:nodes) {
- writer.writeStartElement(element.getLocalName());
- writer.writeCharacters(node.asString());
- writer.writeEndElement();
- }
- }
-
private void writeAttribute(final XMLExtendedStreamWriter writer, final Element element, final ModelNode node) throws XMLStreamException {
if (has(node, element.getModelName())) {
- writer.writeAttribute(element.getLocalName(), node.get(element.getModelName()).asString());
+ String value = node.get(element.getModelName()).asString();
+ if (!element.sameAsDefault(value)) {
+ writer.writeAttribute(element.getLocalName(), value);
+ }
}
}
@@ -298,12 +273,31 @@
Element element = Element.forName(reader.getLocalName());
switch (element) {
case ALLOW_ENV_FUNCTION_ELEMENT:
+ case EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT:
+ case DETECTING_CHANGE_EVENTS_ELEMENT:
bootServices.get(reader.getLocalName()).set(Boolean.parseBoolean(reader.getElementText()));
break;
+
+ case MAX_THREADS_ELEMENT:
+ case MAX_ACTIVE_PLANS_ELEMENT:
+ case USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT:
+ case TIME_SLICE_IN_MILLI_ELEMENT:
+ case MAX_ROWS_FETCH_SIZE_ELEMENT:
+ case LOB_CHUNK_SIZE_IN_KB_ELEMENT:
+ case QUERY_THRESHOLD_IN_SECS_ELEMENT:
+ case MAX_SOURCE_ROWS_ELEMENT:
+ bootServices.get(reader.getLocalName()).set(Integer.parseInt(reader.getElementText()));
+ break;
- case AUTHORIZATION_VALIDATOR_MODULE_ELEMENT:
- case POLICY_DECIDER_MODULE_ELEMENT:
- case ASYNC_THREAD_GROUP_ELEMENT:
+ case AUTHORIZATION_VALIDATOR_ELEMENT:
+ parseAuthorizationValidator(reader, bootServices);
+ break;
+
+ case POLICY_DECIDER_ELEMENT:
+ parsePolicyDecider(reader, bootServices);
+ break;
+
+ case ASYNC_THREAD_POOL_ELEMENT:
bootServices.get(reader.getLocalName()).set(reader.getElementText());
break;
@@ -313,27 +307,33 @@
break;
case BUFFER_SERVICE_ELEMENT:
- parseBufferConfiguration(reader, bootServices);
+ parseBufferService(reader, bootServices);
break;
+
case PREPAREDPLAN_CACHE_ELEMENT:
parsePreparedPlanCacheConfiguration(reader, bootServices);
break;
+
case RESULTSET_CACHE_ELEMENT:
parseResultsetCacheConfiguration(reader, bootServices);
break;
- case QUERY_ENGINE_ELEMENT:
- ModelNode engineNode = new ModelNode();
+ case TRANSPORT_ELEMENT:
+ ModelNode transport = new ModelNode();
- String name = parseQueryEngine(reader, engineNode);
-
- final ModelNode engineAddress = address.clone();
- engineAddress.add(Configuration.QUERY_ENGINE, name);
- engineAddress.protect();
- engineNode.get(OP).set(ADD);
- engineNode.get(OP_ADDR).set(engineAddress);
-
- list.add(engineNode);
+ String name = parseTransport(reader, transport);
+ if (name != null) {
+ final ModelNode transportAddress = address.clone();
+ transportAddress.add("transport", name); //$NON-NLS-1$
+ transportAddress.protect();
+ transport.get(OP).set(ADD);
+ transport.get(OP_ADDR).set(transportAddress);
+
+ list.add(transport);
+ }
+ else {
+ throw new XMLStreamException();
+ }
break;
case TRANSLATOR_ELEMENT:
@@ -341,14 +341,18 @@
String translatorName = parseTranslator(reader, translatorNode);
- final ModelNode translatorAddress = address.clone();
- translatorAddress.add(Configuration.TRANSLATOR, translatorName);
- translatorAddress.protect();
-
- translatorNode.get(OP).set(ADD);
- translatorNode.get(OP_ADDR).set(translatorAddress);
-
- list.add(translatorNode);
+ if (translatorName != null) {
+ final ModelNode translatorAddress = address.clone();
+ translatorAddress.add("translator", translatorName); //$NON-NLS-1$
+ translatorAddress.protect();
+ translatorNode.get(OP).set(ADD);
+ translatorNode.get(OP_ADDR).set(translatorAddress);
+
+ list.add(translatorNode);
+ }
+ else {
+ throw new XMLStreamException();
+ }
break;
default:
@@ -363,22 +367,62 @@
}
}
+ private ModelNode parseAuthorizationValidator(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ if (reader.getAttributeCount() > 0) {
+ for(int i=0; i<reader.getAttributeCount(); i++) {
+ String attrName = reader.getAttributeLocalName(i);
+ String attrValue = reader.getAttributeValue(i);
+
+ Element element = Element.forName(attrName, Element.AUTHORIZATION_VALIDATOR_ELEMENT);
+ switch(element) {
+ case AUTHORIZATION_VALIDATOR_MODULE_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
+ }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
+ return node;
+ }
+
+ private ModelNode parsePolicyDecider(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ if (reader.getAttributeCount() > 0) {
+ for(int i=0; i<reader.getAttributeCount(); i++) {
+ String attrName = reader.getAttributeLocalName(i);
+ String attrValue = reader.getAttributeValue(i);
+
+ Element element = Element.forName(attrName, Element.POLICY_DECIDER_ELEMENT);
+ switch(element) {
+ case POLICY_DECIDER_MODULE_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
+ }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
+ return node;
+ }
+
private ModelNode parseObjectReplicator(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
if (reader.getAttributeCount() > 0) {
for(int i=0; i<reader.getAttributeCount(); i++) {
String attrName = reader.getAttributeLocalName(i);
String attrValue = reader.getAttributeValue(i);
- Element element = Element.forName(attrName, Configuration.OBJECT_REPLICATOR);
+ Element element = Element.forName(attrName, Element.OBJECT_REPLICATOR_ELEMENT);
switch(element) {
case OR_STACK_ATTRIBUTE:
- node.get(Element.OR_STACK_ATTRIBUTE.getModelName()).set(attrValue);
+ node.get(element.getModelName()).set(attrValue);
break;
case OR_CLUSTER_NAME_ATTRIBUTE:
- node.get(Element.OR_CLUSTER_NAME_ATTRIBUTE.getModelName()).set(attrValue);
+ node.get(element.getModelName()).set(attrValue);
break;
default:
- throw ParseUtils.unexpectedElement(reader);
+ throw ParseUtils.unexpectedAttribute(reader, i);
}
}
}
@@ -386,312 +430,296 @@
return node;
}
- private String parseQueryEngine(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
- String engineName = "default"; //$NON-NLS-1$
-
+ private String parseTransport(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ String transportName = null;
if (reader.getAttributeCount() > 0) {
for(int i=0; i<reader.getAttributeCount(); i++) {
String attrName = reader.getAttributeLocalName(i);
String attrValue = reader.getAttributeValue(i);
- if (attrName.equals(Element.ENGINE_NAME_ATTRIBUTE.getLocalName())) {
- engineName = attrValue;
+ Element element = Element.forName(attrName);
+ switch(element) {
+ case TRANSPORT_NAME_ATTRIBUTE:
+ case TRANSLATOR_NAME_ATTRIBUTE:
+ transportName = attrValue;
+ break;
+ case TRANSPORT_SOCKET_BINDING_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+ case TRANSPORT_PROTOCOL_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+ case TRANSPORT_MAX_SOCKET_THREADS_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+ case TRANSPORT_IN_BUFFER_SIZE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+ case TRANSPORT_OUT_BUFFER_SIZE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
}
- else {
- node.get(attrName).set(attrValue);
- }
}
}
while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
Element element = Element.forName(reader.getLocalName());
switch (element) {
- // integers
- case MAX_THREADS_ELEMENT:
- case MAX_ACTIVE_PLANS_ELEMENT:
- case USER_REQUEST_SOURCE_CONCURRENCY_ELEMENT:
- case TIME_SLICE_IN_MILLI_ELEMENT:
- case MAX_ROWS_FETCH_SIZE_ELEMENT:
- case LOB_CHUNK_SIZE_IN_KB_ELEMENT:
- case QUERY_THRESHOLD_IN_SECS_ELEMENT:
- case MAX_SOURCE_ROWS_ELEMENT:
- case MAX_ODBC_LOB_SIZE_ALLOWED_ELEMENT:
- case MAX_SESSIONS_ALLOWED_ELEMENT:
- case SESSION_EXPIRATION_TIME_LIMIT_ELEMENT:
- node.get(reader.getLocalName()).set(Integer.parseInt(reader.getElementText()));
+ case AUTHENTICATION_ELEMENT:
+ parseAuthentication(reader, node);
break;
-
- // booleans
- case EXCEPTION_ON_MAX_SOURCE_ROWS_ELEMENT:
- case DETECTING_CHANGE_EVENTS_ELEMENT:
- node.get(reader.getLocalName()).set(Boolean.parseBoolean(reader.getElementText()));
+ case PG_ELEMENT:
+ parsePg(reader, node);
+ break;
+ case SSL_ELEMENT:
+ parseSSL(reader, node);
break;
-
- //List
- case SECURITY_DOMAIN_ELEMENT:
- node.get(reader.getLocalName()).add(reader.getElementText());
- break;
- case JDBC_ELEMENT:
- parseJDBCSocketConfiguration(reader, node);
- break;
-
- case ODBC_ELEMENT:
- parseODBCSocketConfiguration(reader, node);
- break;
-
default:
throw ParseUtils.unexpectedElement(reader);
}
}
- return engineName;
+ return transportName;
}
-
- private ModelNode parseBufferConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
- while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
- Element element = Element.forName(reader.getLocalName(), Configuration.BUFFER_SERVICE);
- switch (element) {
- case USE_DISK_ELEMENT:
- node.get(Element.USE_DISK_ELEMENT.getModelName()).set(Boolean.parseBoolean(reader.getElementText()));
- break;
- case PROCESSOR_BATCH_SIZE_ELEMENT:
- node.get(Element.PROCESSOR_BATCH_SIZE_ELEMENT.getModelName()).set(Integer.parseInt(reader.getElementText()));
- break;
- case CONNECTOR_BATCH_SIZE_ELEMENT:
- node.get(Element.CONNECTOR_BATCH_SIZE_ELEMENT.getModelName()).set(Integer.parseInt(reader.getElementText()));
- break;
- case MAX_PROCESSING_KB_ELEMENT:
- node.get(Element.MAX_PROCESSING_KB_ELEMENT.getModelName()).set(Integer.parseInt(reader.getElementText()));
- break;
- case MAX_RESERVED_KB_ELEMENT:
- node.get(Element.MAX_RESERVED_KB_ELEMENT.getModelName()).set(Integer.parseInt(reader.getElementText()));
- break;
- case MAX_OPEN_FILES_ELEMENT:
- node.get(Element.MAX_OPEN_FILES_ELEMENT.getModelName()).set(Integer.parseInt(reader.getElementText()));
- break;
- case MAX_FILE_SIZE_ELEMENT:
- node.get(Element.MAX_FILE_SIZE_ELEMENT.getModelName()).set(Long.parseLong(reader.getElementText()));
- break;
- case MAX_BUFFER_SPACE_ELEMENT:
- node.get(Element.MAX_BUFFER_SPACE_ELEMENT.getModelName()).set(Long.parseLong(reader.getElementText()));
- break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
- }
+
+ private ModelNode parseAuthentication(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ if (reader.getAttributeCount() > 0) {
+ for(int i=0; i<reader.getAttributeCount(); i++) {
+ String attrName = reader.getAttributeLocalName(i);
+ String attrValue = reader.getAttributeValue(i);
+ Element element = Element.forName(attrName, Element.AUTHENTICATION_ELEMENT);
+
+ switch(element) {
+ case AUTHENTICATION_SECURITY_DOMAIN_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+
+ case AUTHENTICATION_MAX_SESSIONS_ALLOWED_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+
+ case AUTHENTICATION_SESSION_EXPIRATION_TIME_LIMIT_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
+ }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
return node;
- }
+ }
- private ModelNode parsePreparedPlanCacheConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ private ModelNode parsePg(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
if (reader.getAttributeCount() > 0) {
for(int i=0; i<reader.getAttributeCount(); i++) {
String attrName = reader.getAttributeLocalName(i);
String attrValue = reader.getAttributeValue(i);
- Element element = Element.forName(attrName, Configuration.PREPAREDPLAN_CACHE);
+ Element element = Element.forName(attrName, Element.PG_ELEMENT);
+
switch(element) {
- case PPC_MAX_ENTRIES_ATTRIBUTE:
- node.get(Element.PPC_MAX_ENTRIES_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
- break;
-
- case PPC_MAX_AGE_IN_SECS_ATTRIBUTE:
- node.get(Element.PPC_MAX_AGE_IN_SECS_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
- break;
-
- case PPC_MAX_STALENESS_ATTRIBUTE:
- node.get(Element.PPC_MAX_STALENESS_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
- break;
- default:
- throw ParseUtils.unexpectedElement(reader);
+ case PG_MAX_LOB_SIZE_ALLOWED_ELEMENT:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
}
}
- }
+ }
while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
return node;
}
+
+ private ModelNode parseSSL(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ if (reader.getAttributeCount() > 0) {
+ for(int i=0; i<reader.getAttributeCount(); i++) {
+ String attrName = reader.getAttributeLocalName(i);
+ String attrValue = reader.getAttributeValue(i);
+ Element element = Element.forName(attrName, Element.SSL_ELEMENT);
+
+ switch(element) {
+ case SSL_MODE_ATTRIBUTE:
+ case SSL_AUTH_MODE_ATTRIBUTE:
+ case SSL_SSL_PROTOCOL_ATTRIBUTE:
+ case SSL_KEY_MANAGEMENT_ALG_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
+ }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
+ Element element = Element.forName(reader.getLocalName());
+ switch (element) {
+ case SSL_KETSTORE_ELEMENT:
+ parseKeystore(reader, node);
+ break;
+ case SSL_TRUSTSTORE_ELEMENT:
+ parseTruststore(reader, node);
+ break;
+
+ default:
+ throw ParseUtils.unexpectedElement(reader);
+ }
+ }
+ return node;
+ }
- private ModelNode parseResultsetCacheConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ private ModelNode parseKeystore(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
if (reader.getAttributeCount() > 0) {
for(int i=0; i<reader.getAttributeCount(); i++) {
String attrName = reader.getAttributeLocalName(i);
String attrValue = reader.getAttributeValue(i);
- Element element = Element.forName(attrName, Configuration.RESULTSET_CACHE);
+ Element element = Element.forName(attrName, Element.SSL_KETSTORE_ELEMENT);
+
switch(element) {
- case RSC_CONTAINER_NAME_ELEMENT:
- node.get(Element.RSC_CONTAINER_NAME_ELEMENT.getModelName()).set(attrValue);
+ case SSL_KETSTORE_NAME_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
break;
- case RSC_ENABLE_ATTRIBUTE:
- node.get(Element.RSC_ENABLE_ATTRIBUTE.getModelName()).set(Boolean.parseBoolean(attrValue));
+
+ case SSL_KETSTORE_PASSWORD_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
break;
- case RSC_MAX_STALENESS_ELEMENT:
- node.get(Element.RSC_MAX_STALENESS_ELEMENT.getModelName()).set(Integer.parseInt(attrValue));
+
+ case SSL_KETSTORE_TYPE_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
break;
- case RSC_NAME_ELEMENT:
- node.get(Element.RSC_NAME_ELEMENT.getModelName()).set(attrValue);
+
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
+ }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
+ return node;
+ }
+
+ private ModelNode parseTruststore(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ if (reader.getAttributeCount() > 0) {
+ for(int i=0; i<reader.getAttributeCount(); i++) {
+ String attrName = reader.getAttributeLocalName(i);
+ String attrValue = reader.getAttributeValue(i);
+ Element element = Element.forName(attrName, Element.SSL_TRUSTSTORE_ELEMENT);
+
+ switch(element) {
+ case SSL_TRUSTSTORE_NAME_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
+
+ case SSL_TRUSTSTORE_PASSWORD_ATTRIBUTE:
+ node.get(element.getModelName()).set(attrValue);
+ break;
+
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
}
}
while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
return node;
- }
+ }
- private ModelNode parseJDBCSocketConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
-
+ private ModelNode parseBufferService(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+
if (reader.getAttributeCount() > 0) {
for(int i=0; i<reader.getAttributeCount(); i++) {
String attrName = reader.getAttributeLocalName(i);
String attrValue = reader.getAttributeValue(i);
-
- Element element = Element.forName(attrName, Configuration.JDBC);
- switch (element) {
- case JDBC_SOCKET_BINDING_ATTRIBUTE:
- node.get(Element.JDBC_SOCKET_BINDING_ATTRIBUTE.getModelName()).set(attrValue);
+ Element element = Element.forName(attrName, Element.BUFFER_SERVICE_ELEMENT);
+
+ switch(element) {
+ case USE_DISK_ATTRIBUTE:
+ node.get(element.getModelName()).set(Boolean.parseBoolean(attrValue));
break;
- case JDBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE:
- node.get(Element.JDBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
+ case PROCESSOR_BATCH_SIZE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
break;
- case JDBC_IN_BUFFER_SIZE_ATTRIBUTE:
- node.get(Element.JDBC_IN_BUFFER_SIZE_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
+ case CONNECTOR_BATCH_SIZE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
break;
- case JDBC_OUT_BUFFER_SIZE_ATTRIBUTE:
- node.get(Element.JDBC_OUT_BUFFER_SIZE_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
+ case MAX_PROCESSING_KB_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
+ case MAX_RESERVED_KB_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+ case MAX_OPEN_FILES_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+ case MAX_FILE_SIZE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Long.parseLong(attrValue));
+ break;
+ case MAX_BUFFER_SPACE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Long.parseLong(attrValue));
+ break;
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
}
- }
-
- while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
- Element element = Element.forName(reader.getLocalName(), Configuration.JDBC);
- switch (element) {
- case JDBC_SSL_ELEMENT:
- parseJDBCSSLConfiguration(reader, node);
- break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
- }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
return node;
}
- private ModelNode parseODBCSocketConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
-
+ private ModelNode parsePreparedPlanCacheConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
if (reader.getAttributeCount() > 0) {
for(int i=0; i<reader.getAttributeCount(); i++) {
String attrName = reader.getAttributeLocalName(i);
String attrValue = reader.getAttributeValue(i);
-
- Element element = Element.forName(attrName, Configuration.ODBC);
- switch (element) {
- case ODBC_SOCKET_BINDING_ATTRIBUTE:
- node.get(Element.ODBC_SOCKET_BINDING_ATTRIBUTE.getModelName()).set(attrValue);
+ Element element = Element.forName(attrName, Element.PREPAREDPLAN_CACHE_ELEMENT);
+ switch(element) {
+ case PPC_MAX_ENTRIES_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+
+ case PPC_MAX_AGE_IN_SECS_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+
+ case PPC_MAX_STALENESS_ATTRIBUTE:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
+ break;
+ default:
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
+ }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
+ return node;
+ }
+
+ private ModelNode parseResultsetCacheConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
+ if (reader.getAttributeCount() > 0) {
+ for(int i=0; i<reader.getAttributeCount(); i++) {
+ String attrName = reader.getAttributeLocalName(i);
+ String attrValue = reader.getAttributeValue(i);
+ Element element = Element.forName(attrName, Element.RESULTSET_CACHE_ELEMENT);
+ switch(element) {
+ case RSC_CONTAINER_NAME_ELEMENT:
+ node.get(element.getModelName()).set(attrValue);
break;
- case ODBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE:
- node.get(Element.ODBC_MAX_SOCKET_THREAD_SIZE_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
+ case RSC_ENABLE_ATTRIBUTE:
+ node.get(element.getModelName()).set(Boolean.parseBoolean(attrValue));
break;
- case ODBC_IN_BUFFER_SIZE_ATTRIBUTE:
- node.get(Element.ODBC_IN_BUFFER_SIZE_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
+ case RSC_MAX_STALENESS_ELEMENT:
+ node.get(element.getModelName()).set(Integer.parseInt(attrValue));
break;
- case ODBC_OUT_BUFFER_SIZE_ATTRIBUTE:
- node.get(Element.ODBC_OUT_BUFFER_SIZE_ATTRIBUTE.getModelName()).set(Integer.parseInt(attrValue));
+ case RSC_NAME_ELEMENT:
+ node.get(element.getModelName()).set(attrValue);
break;
default:
- throw ParseUtils.unexpectedElement(reader);
- }
+ throw ParseUtils.unexpectedAttribute(reader, i);
+ }
}
- }
-
- while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
- Element element = Element.forName(reader.getLocalName(), Configuration.ODBC);
- switch (element) {
- case ODBC_SSL_ELEMENT:
- parseODBCSSLConfiguration(reader, node);
- break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
- }
+ }
+ while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT));
return node;
- }
+ }
- private ModelNode parseJDBCSSLConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
- while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
- Element element = Element.forName(reader.getLocalName(), Configuration.JDBC, Configuration.SSL);
- switch (element) {
- case JDBC_SSL_MODE_ELEMENT:
- node.get(Element.JDBC_SSL_MODE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_KEY_STORE_FILE_ELEMENT:
- node.get(Element.JDBC_KEY_STORE_FILE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_KEY_STORE_PASSWD_ELEMENT:
- node.get(Element.JDBC_KEY_STORE_PASSWD_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_KEY_STORE_TYPE_ELEMENT:
- node.get(Element.JDBC_KEY_STORE_TYPE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_SSL_PROTOCOL_ELEMENT:
- node.get(Element.JDBC_SSL_PROTOCOL_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_TRUST_FILE_ELEMENT:
- node.get(Element.JDBC_TRUST_FILE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_TRUST_PASSWD_ELEMENT:
- node.get(Element.JDBC_TRUST_PASSWD_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_AUTH_MODE_ELEMENT:
- node.get(Element.JDBC_AUTH_MODE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case JDBC_KEY_MANAGEMENT_ALG_ELEMENT:
- node.get(Element.JDBC_KEY_MANAGEMENT_ALG_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
- }
- return node;
- }
-
- private ModelNode parseODBCSSLConfiguration(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
- while (reader.hasNext() && (reader.nextTag() != XMLStreamConstants.END_ELEMENT)) {
- Element element = Element.forName(reader.getLocalName(), Configuration.ODBC, Configuration.SSL);
- switch (element) {
- case ODBC_SSL_MODE_ELEMENT:
- node.get(Element.ODBC_SSL_MODE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_KEY_STORE_FILE_ELEMENT:
- node.get(Element.ODBC_KEY_STORE_FILE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_KEY_STORE_PASSWD_ELEMENT:
- node.get(Element.ODBC_KEY_STORE_PASSWD_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_KEY_STORE_TYPE_ELEMENT:
- node.get(Element.ODBC_KEY_STORE_TYPE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_SSL_PROTOCOL_ELEMENT:
- node.get(Element.ODBC_SSL_PROTOCOL_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_TRUST_FILE_ELEMENT:
- node.get(Element.ODBC_TRUST_FILE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_TRUST_PASSWD_ELEMENT:
- node.get(Element.ODBC_TRUST_PASSWD_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_AUTH_MODE_ELEMENT:
- node.get(Element.ODBC_AUTH_MODE_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- case ODBC_KEY_MANAGEMENT_ALG_ELEMENT:
- node.get(Element.ODBC_KEY_MANAGEMENT_ALG_ELEMENT.getModelName()).set(reader.getElementText());
- break;
- default:
- throw ParseUtils.unexpectedElement(reader);
- }
- }
- return node;
- }
-
private String parseTranslator(XMLExtendedStreamReader reader, ModelNode node) throws XMLStreamException {
String translatorName = null;
if (reader.getAttributeCount() > 0) {
@@ -701,14 +729,15 @@
Element element = Element.forName(attrName);
switch(element) {
+ case TRANSPORT_NAME_ATTRIBUTE:
case TRANSLATOR_NAME_ATTRIBUTE:
translatorName = attrValue;
break;
case TRANSLATOR_MODULE_ATTRIBUTE:
- node.get(attrName).set(attrValue);
+ node.get(element.getModelName()).set(attrValue);
break;
default:
- throw ParseUtils.unexpectedElement(reader);
+ throw ParseUtils.unexpectedAttribute(reader, i);
}
}
}
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TranslatorAdd.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TranslatorAdd.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TranslatorAdd.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -21,18 +21,24 @@
*/
package org.teiid.jboss;
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-import static org.teiid.jboss.Configuration.addAttribute;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.ADD;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OPERATION_NAME;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OP_ADDR;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REQUEST_PROPERTIES;
import java.util.List;
import java.util.Locale;
import java.util.ResourceBundle;
import java.util.ServiceLoader;
-import org.jboss.as.controller.*;
+import org.jboss.as.controller.AbstractAddStepHandler;
+import org.jboss.as.controller.OperationContext;
+import org.jboss.as.controller.OperationFailedException;
+import org.jboss.as.controller.PathAddress;
+import org.jboss.as.controller.ServiceVerificationHandler;
import org.jboss.as.controller.descriptions.DescriptionProvider;
import org.jboss.dmr.ModelNode;
-import org.jboss.dmr.ModelType;
import org.jboss.modules.Module;
import org.jboss.modules.ModuleIdentifier;
import org.jboss.modules.ModuleLoadException;
@@ -55,7 +61,7 @@
operation.get(OPERATION_NAME).set(ADD);
operation.get(DESCRIPTION).set(bundle.getString("translator.add")); //$NON-NLS-1$
- addAttribute(operation, Configuration.TRANSLATOR_MODULE, REQUEST_PROPERTIES, bundle.getString(Configuration.TRANSLATOR_MODULE+Configuration.DESC), ModelType.STRING, true, null);
+ Element.TRANSLATOR_MODULE_ATTRIBUTE.describe(operation, REQUEST_PROPERTIES, bundle);
return operation;
}
@@ -65,8 +71,7 @@
}
static void populate(ModelNode operation, ModelNode model) {
- final String moduleName = operation.require(Configuration.TRANSLATOR_MODULE).asString();
- model.get(Configuration.TRANSLATOR_MODULE).set(moduleName);
+ Element.TRANSLATOR_MODULE_ATTRIBUTE.populate(operation, model);
}
@Override
@@ -78,7 +83,7 @@
final String translatorName = pathAddress.getLastElement().getValue();
- final String moduleName = operation.require(Configuration.TRANSLATOR_MODULE).asString();
+ final String moduleName = Element.TRANSLATOR_MODULE_ATTRIBUTE.asString(operation);
final ServiceTarget target = context.getServiceTarget();
@@ -104,7 +109,7 @@
TranslatorService translatorService = new TranslatorService(metadata);
ServiceBuilder<VDBTranslatorMetaData> builder = target.addService(TeiidServiceNames.translatorServiceName(metadata.getName()), translatorService);
builder.addDependency(TeiidServiceNames.TRANSLATOR_REPO, TranslatorRepository.class, translatorService.repositoryInjector);
- builder.setInitialMode(ServiceController.Mode.ACTIVE).install();
+ newControllers.add(builder.setInitialMode(ServiceController.Mode.ACTIVE).install());
added = true;
}
}
Added: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Transport.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Transport.java (rev 0)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Transport.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -0,0 +1,339 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+package org.teiid.jboss;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.jboss.as.network.SocketBinding;
+import org.jboss.as.security.plugins.SecurityDomainContext;
+import org.jboss.msc.service.Service;
+import org.jboss.msc.service.StartContext;
+import org.jboss.msc.service.StartException;
+import org.jboss.msc.service.StopContext;
+import org.jboss.msc.value.InjectedValue;
+import org.teiid.adminapi.AdminComponentException;
+import org.teiid.adminapi.AdminException;
+import org.teiid.adminapi.impl.RequestMetadata;
+import org.teiid.adminapi.impl.SessionMetadata;
+import org.teiid.client.DQP;
+import org.teiid.client.security.ILogon;
+import org.teiid.client.util.ExceptionUtil;
+import org.teiid.core.ComponentNotFoundException;
+import org.teiid.core.util.LRUCache;
+import org.teiid.deployers.CompositeVDB;
+import org.teiid.deployers.VDBLifeCycleListener;
+import org.teiid.deployers.VDBRepository;
+import org.teiid.dqp.internal.process.DQPCore;
+import org.teiid.dqp.internal.process.DQPWorkContext;
+import org.teiid.dqp.internal.process.SessionAwareCache;
+import org.teiid.dqp.service.SessionServiceException;
+import org.teiid.logging.LogConstants;
+import org.teiid.logging.LogManager;
+import org.teiid.logging.MessageLevel;
+import org.teiid.security.SecurityHelper;
+import org.teiid.services.BufferServiceImpl;
+import org.teiid.services.SessionServiceImpl;
+import org.teiid.transport.ClientServiceRegistry;
+import org.teiid.transport.ClientServiceRegistryImpl;
+import org.teiid.transport.LocalServerConnection;
+import org.teiid.transport.LogonImpl;
+import org.teiid.transport.ODBCSocketListener;
+import org.teiid.transport.SocketConfiguration;
+import org.teiid.transport.SocketListener;
+import org.teiid.vdb.runtime.VDBKey;
+
+public class Transport implements Service<ClientServiceRegistry>, ClientServiceRegistry {
+ private enum Protocol {teiid, pg};
+ private ClientServiceRegistryImpl csr = new ClientServiceRegistryImpl();
+ private transient ILogon logon;
+ private SocketConfiguration socketConfig;
+ final ConcurrentMap<String, SecurityDomainContext> securityDomains = new ConcurrentHashMap<String, SecurityDomainContext>();
+ private List<String> authenticationDomains;;
+ private long sessionMaxLimit;
+ private long sessionExpirationTimeLimit;
+ private SocketListener socketListener;
+ private transient SessionServiceImpl sessionService;
+ private String authenticationType;
+ private int maxODBCLobSizeAllowed = 5*1024*1024; // 5 MB
+ private boolean embedded;
+
+ private final InjectedValue<SocketBinding> socketBindingInjector = new InjectedValue<SocketBinding>();
+ private final InjectedValue<VDBRepository> vdbRepositoryInjector = new InjectedValue<VDBRepository>();
+ private final InjectedValue<SessionAwareCache> preparedPlanCacheInjector = new InjectedValue<SessionAwareCache>();
+ private final InjectedValue<SessionAwareCache> resultSetCacheInjector = new InjectedValue<SessionAwareCache>();
+ private final InjectedValue<DQPCore> dqpInjector = new InjectedValue<DQPCore>();
+ private final InjectedValue<BufferServiceImpl> bufferServiceInjector = new InjectedValue<BufferServiceImpl>();
+
+ @Override
+ public <T> T getClientService(Class<T> iface) throws ComponentNotFoundException {
+ return csr.getClientService(iface);
+ }
+
+ @Override
+ public SecurityHelper getSecurityHelper() {
+ return csr.getSecurityHelper();
+ }
+
+ @Override
+ public ClientServiceRegistry getValue() throws IllegalStateException, IllegalArgumentException {
+ return this;
+ }
+
+ @Override
+ public void start(StartContext context) throws StartException {
+ this.csr.setSecurityHelper(new JBossSecurityHelper());
+
+ this.sessionService = new SessionServiceImpl();
+ if (this.authenticationDomains != null && !this.authenticationDomains.isEmpty()) {
+ this.sessionService.setSecurityDomains(this.authenticationDomains, this.securityDomains);
+ }
+ this.sessionService.setSessionExpirationTimeLimit(this.sessionExpirationTimeLimit);
+ this.sessionService.setSessionMaxLimit(this.sessionMaxLimit);
+ this.sessionService.setDqp(getDQP());
+ this.sessionService.setVDBRepository(getVdbRepository());
+ this.sessionService.setSecurityHelper(this.csr.getSecurityHelper());
+ this.sessionService.start();
+
+ // create the necessary services
+ this.logon = new LogonImpl(this.sessionService, "teiid-cluster"); //$NON-NLS-1$
+
+ if (this.socketConfig != null) {
+ InetSocketAddress address = getSocketBindingInjector().getValue().getSocketAddress();
+ Protocol protocol = Protocol.valueOf(socketConfig.getProtocol());
+ if (protocol == Protocol.teiid) {
+ this.socketListener = new SocketListener(address, this.socketConfig, this.csr, getBufferServiceInjector().getValue().getBufferManager());
+ LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("socket_enabled","Teiid JDBC = ",(this.socketConfig.getSSLConfiguration().isSslEnabled()?"mms://":"mm://")+address.getHostName()+":"+address.getPort())); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
+ }
+ else if (protocol == Protocol.pg) {
+ getVdbRepository().odbcEnabled();
+
+ ODBCSocketListener odbc = new ODBCSocketListener(address, this.socketConfig, this.csr, getBufferServiceInjector().getValue().getBufferManager(), getMaxODBCLobSizeAllowed(), this.logon);
+ if (getAuthenticationType() != null) {
+ this.sessionService.setAuthenticationType(getAuthenticationType());
+ odbc.setAuthenticationType(this.sessionService.getAuthType());
+ }
+ LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("odbc_enabled","Teiid ODBC - SSL=", (this.socketConfig.getSSLConfiguration().isSslEnabled()?"ON":"OFF")+" Host = "+address.getHostName()+" Port = "+address.getPort())); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$ //$NON-NLS-6$
+ }
+ else {
+ throw new StartException(IntegrationPlugin.Util.getString("wrong_protocol")); //$NON-NLS-1$
+ }
+ }
+ else {
+ LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("embedded_enabled", LocalServerConnection.TEIID_RUNTIME_CONTEXT)); //$NON-NLS-1$
+ }
+
+ DQP dqpProxy = proxyService(DQP.class, getDQP(), LogConstants.CTX_DQP);
+ this.csr.registerClientService(ILogon.class, logon, LogConstants.CTX_SECURITY);
+ this.csr.registerClientService(DQP.class, dqpProxy, LogConstants.CTX_DQP);
+
+ // add vdb life cycle listeners
+ getVdbRepository().addListener(new VDBLifeCycleListener() {
+
+ private Set<VDBKey> recentlyRemoved = Collections.newSetFromMap(new LRUCache<VDBKey, Boolean>(10000));
+
+ @Override
+ public void removed(String name, int version, CompositeVDB vdb) {
+ recentlyRemoved.add(new VDBKey(name, version));
+ }
+
+ @Override
+ public void added(String name, int version, CompositeVDB vdb) {
+ if (!recentlyRemoved.remove(new VDBKey(name, version))) {
+ return;
+ }
+ // terminate all the previous sessions
+ try {
+ Collection<SessionMetadata> sessions = sessionService.getActiveSessions();
+ for (SessionMetadata session:sessions) {
+ if (name.equalsIgnoreCase(session.getVDBName()) && version == session.getVDBVersion()){
+ sessionService.terminateSession(session.getSessionId(), null);
+ }
+ }
+ } catch (SessionServiceException e) {
+ //ignore
+ }
+
+ // dump the caches.
+ getResultSetCacheInjector().getValue().clearForVDB(name, version);
+ getPreparedPlanCacheInjector().getValue().clearForVDB(name, version);
+ }
+ });
+ }
+
+ @Override
+ public void stop(StopContext context) {
+ // Stop socket transport(s)
+ if (this.socketListener != null) {
+ this.socketListener.stop();
+ this.socketListener = null;
+ }
+ this.sessionService.stop();
+ }
+
+ /**
+ * Creates an proxy to validate the incoming session
+ */
+ private <T> T proxyService(final Class<T> iface, final T instance, String context) {
+
+ return iface.cast(Proxy.newProxyInstance(this.getClass().getClassLoader(), new Class[] {iface}, new LogManager.LoggingProxy(instance, context, MessageLevel.TRACE) {
+
+ public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+ Throwable exception = null;
+ try {
+ sessionService.validateSession(DQPWorkContext.getWorkContext().getSessionId());
+ return super.invoke(proxy, method, args);
+ } catch (InvocationTargetException e) {
+ exception = e.getTargetException();
+ } catch(Throwable t){
+ exception = t;
+ }
+ throw ExceptionUtil.convertException(method, exception);
+ }
+ }));
+ }
+
+ public List<RequestMetadata> getRequestsUsingVDB(String vdbName, int vdbVersion) throws AdminException {
+ List<RequestMetadata> requests = new ArrayList<RequestMetadata>();
+ try {
+ Collection<SessionMetadata> sessions = this.sessionService.getActiveSessions();
+ for (SessionMetadata session:sessions) {
+ if (session.getVDBName().equals(vdbName) && session.getVDBVersion() == vdbVersion) {
+ requests.addAll(getDQP().getRequestsForSession(session.getSessionId()));
+ }
+ }
+ } catch (SessionServiceException e) {
+ throw new AdminComponentException(e);
+ }
+ return requests;
+ }
+
+ public void terminateSession(String terminateeId) {
+ this.sessionService.terminateSession(terminateeId, DQPWorkContext.getWorkContext().getSessionId());
+ }
+
+ public Collection<SessionMetadata> getActiveSessions() throws AdminException {
+ try {
+ return this.sessionService.getActiveSessions();
+ } catch (SessionServiceException e) {
+ throw new AdminComponentException(e);
+ }
+ }
+
+ public int getActiveSessionsCount() throws AdminException{
+ try {
+ return this.sessionService.getActiveSessionsCount();
+ } catch (SessionServiceException e) {
+ throw new AdminComponentException(e);
+ }
+ }
+
+ public InjectedValue<SocketBinding> getSocketBindingInjector() {
+ return this.socketBindingInjector;
+ }
+
+ public SocketConfiguration getSocketConfig() {
+ return socketConfig;
+ }
+
+ public void setSocketConfig(SocketConfiguration socketConfig) {
+ this.socketConfig = socketConfig;
+ }
+
+ public List<String> getAuthenticationDomains() {
+ return authenticationDomains;
+ }
+
+ public void setAuthenticationDomains(List<String> authenticationDomains) {
+ this.authenticationDomains = new LinkedList(authenticationDomains);
+ }
+
+ public void setSessionMaxLimit(long limit) {
+ this.sessionMaxLimit = limit;
+ }
+
+ public void setSessionExpirationTimeLimit(long limit) {
+ this.sessionExpirationTimeLimit = limit;
+ }
+
+ public String getAuthenticationType() {
+ return authenticationType;
+ }
+
+ public void setAuthenticationType(String authenticationType) {
+ this.authenticationType = authenticationType;
+ }
+
+ public InjectedValue<VDBRepository> getVdbRepositoryInjector() {
+ return vdbRepositoryInjector;
+ }
+
+ private VDBRepository getVdbRepository() {
+ return vdbRepositoryInjector.getValue();
+ }
+
+ public InjectedValue<SessionAwareCache> getResultSetCacheInjector() {
+ return resultSetCacheInjector;
+ }
+
+ public InjectedValue<SessionAwareCache> getPreparedPlanCacheInjector() {
+ return preparedPlanCacheInjector;
+ }
+
+ private DQPCore getDQP() {
+ return getDqpInjector().getValue();
+ }
+
+ public InjectedValue<DQPCore> getDqpInjector() {
+ return dqpInjector;
+ }
+
+ public InjectedValue<BufferServiceImpl> getBufferServiceInjector() {
+ return bufferServiceInjector;
+ }
+
+ private int getMaxODBCLobSizeAllowed() {
+ return this.maxODBCLobSizeAllowed;
+ }
+
+ public void setMaxODBCLobSizeAllowed(int lobSize) {
+ this.maxODBCLobSizeAllowed = lobSize;
+ }
+
+ public void setEmbedded(boolean v) {
+ this.embedded = v;
+ }
+
+ public boolean isEmbedded() {
+ return this.embedded;
+ }
+}
Property changes on: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/Transport.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Added: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportAdd.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportAdd.java (rev 0)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportAdd.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -0,0 +1,285 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+package org.teiid.jboss;
+
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.ADD;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OPERATION_NAME;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OP_ADDR;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REQUEST_PROPERTIES;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.ResourceBundle;
+import java.util.StringTokenizer;
+
+import javax.naming.InitialContext;
+import javax.naming.NamingException;
+
+import org.jboss.as.controller.AbstractAddStepHandler;
+import org.jboss.as.controller.OperationContext;
+import org.jboss.as.controller.OperationFailedException;
+import org.jboss.as.controller.PathAddress;
+import org.jboss.as.controller.ServiceVerificationHandler;
+import org.jboss.as.controller.descriptions.DescriptionProvider;
+import org.jboss.as.naming.ManagedReferenceFactory;
+import org.jboss.as.naming.NamingStore;
+import org.jboss.as.naming.deployment.ContextNames;
+import org.jboss.as.naming.service.BinderService;
+import org.jboss.as.network.SocketBinding;
+import org.jboss.as.security.plugins.SecurityDomainContext;
+import org.jboss.dmr.ModelNode;
+import org.jboss.msc.inject.ConcurrentMapInjector;
+import org.jboss.msc.service.ServiceBuilder;
+import org.jboss.msc.service.ServiceController;
+import org.jboss.msc.service.ServiceName;
+import org.jboss.msc.service.ServiceTarget;
+import org.teiid.deployers.VDBRepository;
+import org.teiid.dqp.internal.process.DQPCore;
+import org.teiid.dqp.internal.process.SessionAwareCache;
+import org.teiid.logging.LogConstants;
+import org.teiid.logging.LogManager;
+import org.teiid.services.BufferServiceImpl;
+import org.teiid.transport.ClientServiceRegistry;
+import org.teiid.transport.LocalServerConnection;
+import org.teiid.transport.SSLConfiguration;
+import org.teiid.transport.SocketConfiguration;
+
+public class TransportAdd extends AbstractAddStepHandler implements DescriptionProvider {
+
+ private static Element[] attributes = {
+ Element.TRANSPORT_PROTOCOL_ATTRIBUTE,
+ Element.TRANSPORT_SOCKET_BINDING_ATTRIBUTE,
+ Element.TRANSPORT_MAX_SOCKET_THREADS_ATTRIBUTE,
+ Element.TRANSPORT_IN_BUFFER_SIZE_ATTRIBUTE,
+ Element.TRANSPORT_OUT_BUFFER_SIZE_ATTRIBUTE,
+
+ Element.AUTHENTICATION_SECURITY_DOMAIN_ATTRIBUTE,
+ Element.AUTHENTICATION_MAX_SESSIONS_ALLOWED_ATTRIBUTE,
+ Element.AUTHENTICATION_SESSION_EXPIRATION_TIME_LIMIT_ATTRIBUTE,
+
+ Element.PG_MAX_LOB_SIZE_ALLOWED_ELEMENT,
+
+ Element.SSL_MODE_ATTRIBUTE,
+ Element.SSL_AUTH_MODE_ATTRIBUTE,
+ Element.SSL_SSL_PROTOCOL_ATTRIBUTE,
+ Element.SSL_KEY_MANAGEMENT_ALG_ATTRIBUTE,
+ Element.SSL_KETSTORE_NAME_ATTRIBUTE,
+ Element.SSL_KETSTORE_PASSWORD_ATTRIBUTE,
+ Element.SSL_KETSTORE_TYPE_ATTRIBUTE,
+ Element.SSL_TRUSTSTORE_NAME_ATTRIBUTE,
+ Element.SSL_TRUSTSTORE_PASSWORD_ATTRIBUTE
+ };
+
+ @Override
+ public ModelNode getModelDescription(Locale locale) {
+ final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
+
+ final ModelNode node = new ModelNode();
+ node.get(OPERATION_NAME).set(ADD);
+ node.get(DESCRIPTION).set("transport.add"); //$NON-NLS-1$
+
+ describeTransport(node, REQUEST_PROPERTIES, bundle);
+ return node;
+ }
+
+ static void describeTransport(ModelNode node, String type, ResourceBundle bundle) {
+ transportDescribe(node, type, bundle);
+ }
+
+ static void transportDescribe(ModelNode node, String type, ResourceBundle bundle) {
+ for (int i = 0; i < attributes.length; i++) {
+ attributes[i].describe(node, type, bundle);
+ }
+ }
+
+ @Override
+ protected void populateModel(ModelNode operation, ModelNode model) {
+ populate(operation, model);
+ }
+
+ public static void populate(ModelNode operation, ModelNode model) {
+ for (int i = 0; i < attributes.length; i++) {
+ attributes[i].populate(operation, model);
+ }
+ }
+
+ @Override
+ protected void performRuntime(final OperationContext context, final ModelNode operation, final ModelNode model,
+ final ServiceVerificationHandler verificationHandler, final List<ServiceController<?>> newControllers) throws OperationFailedException {
+
+ ServiceTarget target = context.getServiceTarget();
+
+ final ModelNode address = operation.require(OP_ADDR);
+ final PathAddress pathAddress = PathAddress.pathAddress(address);
+ final String transportName = pathAddress.getLastElement().getValue();
+
+ Transport transport = new Transport();
+
+ String socketBinding = null;
+ if (Element.TRANSPORT_SOCKET_BINDING_ATTRIBUTE.isDefined(operation)) {
+ socketBinding = Element.TRANSPORT_SOCKET_BINDING_ATTRIBUTE.asString(operation);
+ transport.setSocketConfig(buildSocketConfiguration(operation));
+ }
+ else {
+ transport.setEmbedded(true);
+ LogManager.logDetail(LogConstants.CTX_SECURITY, IntegrationPlugin.Util.getString("socket_binding_not_defined", transportName)); //$NON-NLS-1$
+ }
+
+ ArrayList<String> domainList = new ArrayList<String>();
+ if (Element.AUTHENTICATION_SECURITY_DOMAIN_ATTRIBUTE.isDefined(operation)) {
+ String domains = Element.AUTHENTICATION_SECURITY_DOMAIN_ATTRIBUTE.asString(operation);
+ StringTokenizer st = new StringTokenizer(domains, ","); //$NON-NLS-1$
+ while(st.hasMoreTokens()) {
+ domainList.add(st.nextToken());
+ }
+ }
+ transport.setAuthenticationDomains(domainList);
+
+ if (Element.AUTHENTICATION_MAX_SESSIONS_ALLOWED_ATTRIBUTE.isDefined(operation)) {
+ transport.setSessionMaxLimit(Element.AUTHENTICATION_MAX_SESSIONS_ALLOWED_ATTRIBUTE.asLong(operation));
+ }
+
+ if (Element.AUTHENTICATION_SESSION_EXPIRATION_TIME_LIMIT_ATTRIBUTE.isDefined(operation)) {
+ transport.setSessionExpirationTimeLimit(Element.AUTHENTICATION_SESSION_EXPIRATION_TIME_LIMIT_ATTRIBUTE.asLong(operation));
+ }
+ if (Element.PG_AUTHENTICATION_TYPE_ATTRIBUTE.isDefined(operation)) {
+ transport.setAuthenticationType(Element.PG_AUTHENTICATION_TYPE_ATTRIBUTE.asString(operation));
+ }
+
+ if (Element.PG_MAX_LOB_SIZE_ALLOWED_ELEMENT.isDefined(operation)) {
+ transport.setMaxODBCLobSizeAllowed(Element.PG_MAX_LOB_SIZE_ALLOWED_ELEMENT.asInt(operation));
+ }
+
+ ServiceBuilder<ClientServiceRegistry> transportBuilder = target.addService(TeiidServiceNames.transportServiceName(transportName), transport);
+ if (socketBinding != null) {
+ transportBuilder.addDependency(ServiceName.JBOSS.append("binding", socketBinding), SocketBinding.class, transport.getSocketBindingInjector()); //$NON-NLS-1$
+ }
+ transportBuilder.addDependency(TeiidServiceNames.BUFFER_MGR, BufferServiceImpl.class, transport.getBufferServiceInjector());
+ transportBuilder.addDependency(TeiidServiceNames.VDB_REPO, VDBRepository.class, transport.getVdbRepositoryInjector());
+ transportBuilder.addDependency(TeiidServiceNames.CACHE_RESULTSET, SessionAwareCache.class, transport.getResultSetCacheInjector());
+ transportBuilder.addDependency(TeiidServiceNames.CACHE_PREPAREDPLAN, SessionAwareCache.class, transport.getPreparedPlanCacheInjector());
+ transportBuilder.addDependency(TeiidServiceNames.ENGINE, DQPCore.class, transport.getDqpInjector());
+
+
+ // add security domains
+ for (String domain:domainList) {
+ LogManager.logInfo(LogConstants.CTX_SECURITY, IntegrationPlugin.Util.getString("security_enabled", domain)); //$NON-NLS-1$
+ transportBuilder.addDependency(ServiceName.JBOSS.append("security", "security-domain", domain), SecurityDomainContext.class, new ConcurrentMapInjector<String,SecurityDomainContext>(transport.securityDomains, domain)); //$NON-NLS-1$ //$NON-NLS-2$
+ }
+
+ transportBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
+ newControllers.add(transportBuilder.install());
+
+ // register a JNDI name, this looks hard.
+ if (transport.isEmbedded() && !isEmbeddedRegistered()) {
+ final ClientServiceRegistryReferenceFactoryService referenceFactoryService = new ClientServiceRegistryReferenceFactoryService();
+ final ServiceName referenceFactoryServiceName =TeiidServiceNames.transportServiceName(transportName).append("reference-factory"); //$NON-NLS-1$
+ final ServiceBuilder<?> referenceBuilder = target.addService(referenceFactoryServiceName,referenceFactoryService);
+ referenceBuilder.addDependency(TeiidServiceNames.transportServiceName(transportName), ClientServiceRegistry.class, referenceFactoryService.getCSRInjector());
+ referenceBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
+
+ final ContextNames.BindInfo bindInfo = ContextNames.bindInfoFor(LocalServerConnection.TEIID_RUNTIME_CONTEXT);
+ final BinderService embedded = new BinderService(bindInfo.getBindName());
+ final ServiceBuilder<?> embeddedBinderBuilder = target.addService(bindInfo.getBinderServiceName(), embedded);
+ embeddedBinderBuilder.addDependency(referenceFactoryServiceName, ManagedReferenceFactory.class, embedded.getManagedObjectInjector());
+ embeddedBinderBuilder.addDependency(bindInfo.getParentContextServiceName(), NamingStore.class, embedded.getNamingStoreInjector());
+ embeddedBinderBuilder.setInitialMode(ServiceController.Mode.ACTIVE);
+
+ newControllers.add(referenceBuilder.install());
+ newControllers.add(embeddedBinderBuilder.install());
+ }
+ }
+
+ protected boolean isEmbeddedRegistered() {
+ try {
+ InitialContext ic = new InitialContext();
+ ic.lookup(LocalServerConnection.TEIID_RUNTIME_CONTEXT);
+ return true;
+ } catch (NamingException e) {
+ return false;
+ }
+ }
+
+ private SocketConfiguration buildSocketConfiguration(ModelNode node) {
+
+ SocketConfiguration socket = new SocketConfiguration();
+
+ if (Element.TRANSPORT_PROTOCOL_ATTRIBUTE.isDefined(node)) {
+ socket.setProtocol(Element.TRANSPORT_PROTOCOL_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.TRANSPORT_MAX_SOCKET_THREADS_ATTRIBUTE.isDefined(node)) {
+ socket.setMaxSocketThreads(Element.TRANSPORT_MAX_SOCKET_THREADS_ATTRIBUTE.asInt(node));
+ }
+
+ if (Element.TRANSPORT_IN_BUFFER_SIZE_ATTRIBUTE.isDefined(node)) {
+ socket.setInputBufferSize(Element.TRANSPORT_IN_BUFFER_SIZE_ATTRIBUTE.asInt(node));
+ }
+
+ if (Element.TRANSPORT_OUT_BUFFER_SIZE_ATTRIBUTE.isDefined(node)) {
+ socket.setOutputBufferSize(Element.TRANSPORT_OUT_BUFFER_SIZE_ATTRIBUTE.asInt(node));
+ }
+
+ SSLConfiguration ssl = new SSLConfiguration();
+ ssl.setAuthenticationMode(SSLConfiguration.ANONYMOUS);
+
+ if (Element.SSL_MODE_ATTRIBUTE.isDefined(node)) {
+ ssl.setMode(Element.SSL_MODE_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_SSL_PROTOCOL_ATTRIBUTE.isDefined(node)) {
+ ssl.setSslProtocol(Element.SSL_SSL_PROTOCOL_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_KEY_MANAGEMENT_ALG_ATTRIBUTE.isDefined(node)) {
+ ssl.setKeymanagementAlgorithm(Element.SSL_KEY_MANAGEMENT_ALG_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_AUTH_MODE_ATTRIBUTE.isDefined(node)) {
+ ssl.setAuthenticationMode(Element.SSL_AUTH_MODE_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_KETSTORE_NAME_ATTRIBUTE.isDefined(node)) {
+ ssl.setKeystoreFilename(Element.SSL_KETSTORE_NAME_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_KETSTORE_PASSWORD_ATTRIBUTE.isDefined(node)) {
+ ssl.setKeystorePassword(Element.SSL_KETSTORE_PASSWORD_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_KETSTORE_TYPE_ATTRIBUTE.isDefined(node)) {
+ ssl.setKeystoreType(Element.SSL_KETSTORE_TYPE_ATTRIBUTE.asString(node));
+ }
+
+ if (Element.SSL_TRUSTSTORE_NAME_ATTRIBUTE.isDefined(node)) {
+ ssl.setTruststoreFilename(Element.SSL_TRUSTSTORE_NAME_ATTRIBUTE.asString(node));
+ }
+ if (Element.SSL_TRUSTSTORE_PASSWORD_ATTRIBUTE.isDefined(node)) {
+ ssl.setTruststorePassword(Element.SSL_TRUSTSTORE_PASSWORD_ATTRIBUTE.asString(node));
+ }
+ socket.setSSLConfiguration(ssl);
+ return socket;
+ }
+
+}
Property changes on: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportAdd.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Copied: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportRemove.java (from rev 3506, branches/as7/jboss-integration/src/main/java/org/teiid/jboss/QueryEngineRemove.java)
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportRemove.java (rev 0)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportRemove.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -0,0 +1,81 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership. Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+package org.teiid.jboss;
+
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIBE;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.DESCRIPTION;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OPERATION_NAME;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.OP_ADDR;
+import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.REMOVE;
+
+import java.util.Locale;
+import java.util.ResourceBundle;
+
+import org.jboss.as.controller.AbstractRemoveStepHandler;
+import org.jboss.as.controller.OperationContext;
+import org.jboss.as.controller.PathAddress;
+import org.jboss.as.controller.descriptions.DescriptionProvider;
+import org.jboss.dmr.ModelNode;
+import org.jboss.msc.service.ServiceController;
+import org.jboss.msc.service.ServiceName;
+import org.jboss.msc.service.ServiceRegistry;
+
+class TransportRemove extends AbstractRemoveStepHandler implements DescriptionProvider {
+
+ @Override
+ protected void performRuntime(OperationContext context, ModelNode operation, ModelNode model) {
+
+ final ModelNode address = operation.require(OP_ADDR);
+ final PathAddress pathAddress = PathAddress.pathAddress(address);
+
+ String transportName = pathAddress.getLastElement().getValue();
+
+ final ServiceRegistry serviceRegistry = context.getServiceRegistry(true);
+ ServiceName serviceName = TeiidServiceNames.transportServiceName(transportName);
+ final ServiceController<?> controller = serviceRegistry.getService(serviceName);
+ if (controller != null) {
+ context.removeService(serviceName);
+ }
+
+// final ServiceName referenceFactoryServiceName = TeiidServiceNames.transportServiceName(transportName).append("reference-factory"); //$NON-NLS-1$
+// final ServiceController<?> referceFactoryController = serviceRegistry.getService(referenceFactoryServiceName);
+// if (referceFactoryController != null) {
+// context.removeService(referenceFactoryServiceName);
+// }
+//
+// final ContextNames.BindInfo bindInfo = ContextNames.bindInfoFor(LocalServerConnection.TEIID_RUNTIME_CONTEXT+transportName);
+// final ServiceController<?> binderController = serviceRegistry.getService(bindInfo.getBinderServiceName());
+// if (binderController != null) {
+// context.removeService(bindInfo.getBinderServiceName());
+// }
+ }
+
+ @Override
+ public ModelNode getModelDescription(Locale locale) {
+ final ResourceBundle bundle = IntegrationPlugin.getResourceBundle(locale);
+ final ModelNode operation = new ModelNode();
+ operation.get(OPERATION_NAME).set(REMOVE);
+ operation.get(DESCRIPTION).set(bundle.getString(REMOVE+"."+DESCRIBE)); //$NON-NLS-1$
+ return operation;
+ }
+
+}
Property changes on: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/TransportRemove.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Modified: branches/as7/jboss-integration/src/main/java/org/teiid/jboss/deployers/RuntimeEngineDeployer.java
===================================================================
--- branches/as7/jboss-integration/src/main/java/org/teiid/jboss/deployers/RuntimeEngineDeployer.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/java/org/teiid/jboss/deployers/RuntimeEngineDeployer.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -23,53 +23,32 @@
import java.io.Serializable;
import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
-import java.net.InetSocketAddress;
-import java.sql.Blob;
-import java.sql.Clob;
-import java.sql.SQLException;
-import java.sql.SQLXML;
-import java.util.*;
-import java.util.concurrent.*;
+import java.util.Date;
+import java.util.List;
import javax.resource.spi.XATerminator;
import javax.resource.spi.work.WorkManager;
-import javax.security.auth.login.LoginException;
import javax.transaction.TransactionManager;
-import org.jboss.as.network.SocketBinding;
-import org.jboss.as.security.plugins.SecurityDomainContext;
import org.jboss.modules.Module;
import org.jboss.msc.service.Service;
import org.jboss.msc.service.StartContext;
import org.jboss.msc.service.StopContext;
import org.jboss.msc.value.InjectedValue;
-import org.teiid.adminapi.AdminComponentException;
-import org.teiid.adminapi.AdminException;
-import org.teiid.adminapi.AdminProcessingException;
-import org.teiid.adminapi.impl.*;
-import org.teiid.client.DQP;
-import org.teiid.client.RequestMessage;
-import org.teiid.client.ResultsMessage;
-import org.teiid.client.security.ILogon;
-import org.teiid.client.security.InvalidSessionException;
-import org.teiid.client.util.ExceptionUtil;
-import org.teiid.client.util.ResultsFuture;
-import org.teiid.core.ComponentNotFoundException;
+import org.teiid.adminapi.impl.VDBMetaData;
import org.teiid.core.TeiidComponentException;
import org.teiid.core.TeiidRuntimeException;
-import org.teiid.core.util.ApplicationInfo;
-import org.teiid.core.util.LRUCache;
-import org.teiid.deployers.CompositeVDB;
-import org.teiid.deployers.VDBLifeCycleListener;
import org.teiid.deployers.VDBRepository;
import org.teiid.dqp.internal.datamgr.TranslatorRepository;
-import org.teiid.dqp.internal.process.*;
+import org.teiid.dqp.internal.process.AuthorizationValidator;
+import org.teiid.dqp.internal.process.DQPConfiguration;
+import org.teiid.dqp.internal.process.DQPCore;
+import org.teiid.dqp.internal.process.DataTierManagerImpl;
+import org.teiid.dqp.internal.process.SessionAwareCache;
+import org.teiid.dqp.internal.process.TransactionServerImpl;
import org.teiid.dqp.service.BufferService;
-import org.teiid.dqp.service.SessionService;
-import org.teiid.dqp.service.SessionServiceException;
import org.teiid.dqp.service.TransactionService;
import org.teiid.events.EventDistributor;
import org.teiid.events.EventDistributorFactory;
@@ -77,48 +56,35 @@
import org.teiid.logging.LogConstants;
import org.teiid.logging.LogManager;
import org.teiid.logging.MessageLevel;
-import org.teiid.metadata.*;
+import org.teiid.metadata.AbstractMetadataRecord;
+import org.teiid.metadata.Column;
+import org.teiid.metadata.ColumnStats;
+import org.teiid.metadata.Procedure;
+import org.teiid.metadata.Schema;
+import org.teiid.metadata.Table;
import org.teiid.metadata.Table.TriggerEvent;
-import org.teiid.net.TeiidURL;
+import org.teiid.metadata.TableStats;
import org.teiid.query.ObjectReplicator;
import org.teiid.query.metadata.TransformationMetadata;
import org.teiid.query.optimizer.relational.RelationalPlanner;
import org.teiid.query.processor.DdlPlan;
import org.teiid.query.tempdata.GlobalTableStore;
-import org.teiid.security.SecurityHelper;
import org.teiid.services.BufferServiceImpl;
-import org.teiid.services.SessionServiceImpl;
-import org.teiid.transport.*;
-import org.teiid.vdb.runtime.VDBKey;
+import org.teiid.transport.LocalServerConnection;
-public class RuntimeEngineDeployer extends DQPConfiguration implements DQPManagement, Serializable , ClientServiceRegistry, EventDistributor, EventDistributorFactory, Service<ClientServiceRegistry> {
+public class RuntimeEngineDeployer extends DQPConfiguration implements Serializable, EventDistributor, EventDistributorFactory, Service<DQPCore> {
private static final long serialVersionUID = -4676205340262775388L;
- private transient SocketConfiguration jdbcSocketConfiguration;
- private transient SocketConfiguration odbcSocketConfiguration;
- private transient SocketListener jdbcSocket;
- private transient SocketListener odbcSocket;
private transient TransactionServerImpl transactionServerImpl = new TransactionServerImpl();
-
private transient DQPCore dqpCore = new DQPCore();
- private transient SessionServiceImpl sessionService;
- private transient ILogon logon;
- private transient ClientServiceRegistryImpl csr = new ClientServiceRegistryImpl();
- private transient VDBRepository vdbRepository;
-
- private long sessionMaxLimit = SessionService.DEFAULT_MAX_SESSIONS;
- private long sessionExpirationTimeLimit = SessionService.DEFAULT_SESSION_EXPIRATION;
-
private transient EventDistributor eventDistributor;
private transient EventDistributor eventDistributorProxy;
private final InjectedValue<WorkManager> workManagerInjector = new InjectedValue<WorkManager>();
private final InjectedValue<XATerminator> xaTerminatorInjector = new InjectedValue<XATerminator>();
private final InjectedValue<TransactionManager> txnManagerInjector = new InjectedValue<TransactionManager>();
- private final InjectedValue<SocketBinding> jdbcSocketBindingInjector = new InjectedValue<SocketBinding>();
private final InjectedValue<BufferServiceImpl> bufferServiceInjector = new InjectedValue<BufferServiceImpl>();
- private final InjectedValue<SocketBinding> odbcSocketBindingInjector = new InjectedValue<SocketBinding>();
private final InjectedValue<TranslatorRepository> translatorRepositoryInjector = new InjectedValue<TranslatorRepository>();
private final InjectedValue<VDBRepository> vdbRepositoryInjector = new InjectedValue<VDBRepository>();
private final InjectedValue<AuthorizationValidator> authorizationValidatorInjector = new InjectedValue<AuthorizationValidator>();
@@ -126,56 +92,21 @@
private final InjectedValue<SessionAwareCache> resultSetCacheInjector = new InjectedValue<SessionAwareCache>();
private final InjectedValue<ObjectReplicator> objectReplicatorInjector = new InjectedValue<ObjectReplicator>();
- public final ConcurrentMap<String, SecurityDomainContext> securityDomains = new ConcurrentHashMap<String, SecurityDomainContext>();
- private LinkedList<String> securityDomainNames = new LinkedList<String>();
- private String instanceName;
-
- public RuntimeEngineDeployer(String name) {
- this.instanceName = name;
- }
-
- public String getName() {
- return this.instanceName;
- }
-
@Override
- public <T> T getClientService(Class<T> iface)
- throws ComponentNotFoundException {
- return this.csr.getClientService(iface);
- }
-
- @Override
- public SecurityHelper getSecurityHelper() {
- return this.csr.getSecurityHelper();
- }
-
- @Override
public void start(StartContext context) {
this.transactionServerImpl.setWorkManager(getWorkManagerInjector().getValue());
this.transactionServerImpl.setXaTerminator(getXaTerminatorInjector().getValue());
this.transactionServerImpl.setTransactionManager(getTxnManagerInjector().getValue());
- setVDBRepository(vdbRepositoryInjector.getValue());
- setAuthorizationValidator(getAuthorizationValidatorInjector().getValue());
+ setAuthorizationValidator(authorizationValidatorInjector.getValue());
- this.sessionService = new SessionServiceImpl();
- if (!this.securityDomainNames.isEmpty()) {
- this.sessionService.setSecurityDomains(this.securityDomainNames, this.securityDomains);
- }
- this.sessionService.setSessionExpirationTimeLimit(this.sessionExpirationTimeLimit);
- this.sessionService.setSessionMaxLimit(this.sessionMaxLimit);
- this.sessionService.setDqp(this.dqpCore);
- this.sessionService.setVDBRepository(this.vdbRepository);
- this.sessionService.setSecurityHelper(getSecurityHelper());
- this.sessionService.start();
+ setBufferService(bufferServiceInjector.getValue());
- this.setBufferService(bufferServiceInjector.getValue());
-
dqpCore.setTransactionService((TransactionService)LogManager.createLoggingProxy(LogConstants.CTX_TXN_LOG, transactionServerImpl, new Class[] {TransactionService.class}, MessageLevel.DETAIL, Module.getCallerModule().getClassLoader()));
if (getObjectReplicatorInjector().getValue() != null) {
try {
- this.eventDistributor = getObjectReplicatorInjector().getValue().replicate(LocalServerConnection.TEIID_RUNTIME_CONTEXT+getName(), EventDistributor.class, this, 0);
+ this.eventDistributor = getObjectReplicatorInjector().getValue().replicate(LocalServerConnection.TEIID_RUNTIME_CONTEXT, EventDistributor.class, this, 0);
} catch (Exception e) {
LogManager.logError(LogConstants.CTX_RUNTIME, e, IntegrationPlugin.Util.getString("replication_failed", this)); //$NON-NLS-1$
}
@@ -184,7 +115,7 @@
LogManager.logDetail(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("distributed_cache_not_enabled")); //$NON-NLS-1$
}
- this.dqpCore.setMetadataRepository(this.vdbRepository.getMetadataRepository());
+ this.dqpCore.setMetadataRepository(getVdbRepository().getMetadataRepository());
this.dqpCore.setEventDistributor(this.eventDistributor);
this.dqpCore.setResultsetCache(getResultSetCacheInjector().getValue());
this.dqpCore.setPreparedPlanCache(getPreparedPlanCacheInjector().getValue());
@@ -201,74 +132,13 @@
return null;
}
});
- // create the necessary services
- createClientServices();
-
- this.csr.registerClientService(ILogon.class, logon, LogConstants.CTX_SECURITY);
- DQP dqpProxy = proxyService(DQP.class, this.dqpCore, LogConstants.CTX_DQP);
- this.csr.registerClientService(DQP.class, dqpProxy, LogConstants.CTX_DQP);
-
- ClientServiceRegistryImpl jdbcCsr = new ClientServiceRegistryImpl();
- jdbcCsr.registerClientService(ILogon.class, logon, LogConstants.CTX_SECURITY);
- jdbcCsr.registerClientService(DQP.class, dqpProxy, LogConstants.CTX_DQP);
-
- if (this.jdbcSocketConfiguration != null) {
- InetSocketAddress address = getJdbcSocketBindingInjector().getValue().getSocketAddress();
- this.jdbcSocket = new SocketListener(address, this.jdbcSocketConfiguration, jdbcCsr, this.dqpCore.getBufferManager());
- LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("socket_enabled","Teiid JDBC = ",(this.jdbcSocketConfiguration.getSSLConfiguration().isSslEnabled()?"mms://":"mm://")+address.getHostName()+":"+address.getPort())); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
- } else {
- LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("socket_not_enabled", "jdbc connections")); //$NON-NLS-1$ //$NON-NLS-2$
- }
-
- if (this.odbcSocketConfiguration != null) {
- this.vdbRepository.odbcEnabled();
- InetSocketAddress address = getOdbcSocketBindingInjector().getValue().getSocketAddress();
- ODBCSocketListener odbc = new ODBCSocketListener(address, this.odbcSocketConfiguration, this.dqpCore.getBufferManager(), getMaxODBCLobSizeAllowed(), this.logon);
- odbc.setAuthenticationType(sessionService.getAuthType());
- this.odbcSocket = odbc;
- LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("odbc_enabled","Teiid ODBC - SSL=", (this.odbcSocketConfiguration.getSSLConfiguration().isSslEnabled()?"ON":"OFF")+" Host = "+address.getHostName()+" Port = "+address.getPort())); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$ //$NON-NLS-6$
- } else {
- LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("odbc_not_enabled")); //$NON-NLS-1$
- }
-
- LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("engine_started", getRuntimeVersion(), new Date(System.currentTimeMillis()).toString())); //$NON-NLS-1$
- // add vdb life cycle listeners
- this.vdbRepository.addListener(new VDBLifeCycleListener() {
-
- private Set<VDBKey> recentlyRemoved = Collections.newSetFromMap(new LRUCache<VDBKey, Boolean>(10000));
-
- @Override
- public void removed(String name, int version, CompositeVDB vdb) {
- recentlyRemoved.add(new VDBKey(name, version));
- }
-
- @Override
- public void added(String name, int version, CompositeVDB vdb) {
- if (!recentlyRemoved.remove(new VDBKey(name, version))) {
- return;
- }
- // terminate all the previous sessions
- try {
- Collection<SessionMetadata> sessions = sessionService.getActiveSessions();
- for (SessionMetadata session:sessions) {
- if (name.equalsIgnoreCase(session.getVDBName()) && version == session.getVDBVersion()){
- sessionService.terminateSession(session.getSessionId(), null);
- }
- }
- } catch (SessionServiceException e) {
- //ignore
- }
- // dump the caches.
- getResultSetCacheInjector().getValue().clearForVDB(name, version);
- getPreparedPlanCacheInjector().getValue().clearForVDB(name, version);
- }
- });
+ LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("engine_started", this.dqpCore.getRuntimeVersion(), new Date(System.currentTimeMillis()).toString())); //$NON-NLS-1$
}
@Override
- public ClientServiceRegistry getValue() throws IllegalStateException, IllegalArgumentException {
- return this;
+ public DQPCore getValue() throws IllegalStateException, IllegalArgumentException {
+ return this.dqpCore;
}
@Override
@@ -279,19 +149,6 @@
// this bean is already shutdown
}
- // Stop socket transport(s)
- if (this.jdbcSocket != null) {
- this.jdbcSocket.stop();
- this.jdbcSocket = null;
- }
-
- if (this.odbcSocket != null) {
- this.odbcSocket.stop();
- this.odbcSocket = null;
- }
-
- this.sessionService.stop();
-
LogManager.logInfo(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("engine_stopped", new Date(System.currentTimeMillis()).toString())); //$NON-NLS-1$
if (getObjectReplicatorInjector().getValue() != null && this.eventDistributor != null) {
@@ -299,259 +156,15 @@
}
}
- private void createClientServices() {
- this.logon = new LogonImpl(this.sessionService, "teiid-cluster"); //$NON-NLS-1$
- }
-
- /**
- * Creates an proxy to validate the incoming session
- */
- private <T> T proxyService(final Class<T> iface, final T instance, String context) {
-
- return iface.cast(Proxy.newProxyInstance(this.getClass().getClassLoader(), new Class[] {iface}, new LogManager.LoggingProxy(instance, context, MessageLevel.TRACE) {
-
- public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
- Throwable exception = null;
- try {
- sessionService.validateSession(DQPWorkContext.getWorkContext().getSessionId());
- return super.invoke(proxy, method, args);
- } catch (InvocationTargetException e) {
- exception = e.getTargetException();
- } catch(Throwable t){
- exception = t;
- }
- throw ExceptionUtil.convertException(method, exception);
- }
- }));
- }
-
- public void setJdbcSocketConfiguration(SocketConfiguration socketConfig) {
- this.jdbcSocketConfiguration = socketConfig;
- }
-
- public void setOdbcSocketConfiguration(SocketConfiguration socketConfig) {
- this.odbcSocketConfiguration = socketConfig;
- }
-
-
public void setBufferService(BufferService service) {
this.dqpCore.setBufferService(service);
}
- public void setSecurityHelper(SecurityHelper helper) {
- this.csr.setSecurityHelper(helper);
- }
- public void setVDBRepository(VDBRepository repo) {
- this.vdbRepository = repo;
- }
-
@Override
- public List<RequestMetadata> getRequestsForSession(String sessionId) {
- return this.dqpCore.getRequestsForSession(sessionId);
- }
-
- @Override
- public List<RequestMetadata> getRequestsUsingVDB(String vdbName, int vdbVersion) throws AdminException {
- List<RequestMetadata> requests = new ArrayList<RequestMetadata>();
- try {
- Collection<SessionMetadata> sessions = this.sessionService.getActiveSessions();
- for (SessionMetadata session:sessions) {
- if (session.getVDBName().equals(vdbName) && session.getVDBVersion() == vdbVersion) {
- requests.addAll(this.dqpCore.getRequestsForSession(session.getSessionId()));
- }
- }
- } catch (SessionServiceException e) {
- throw new AdminComponentException(e);
- }
- return requests;
- }
-
-
- @Override
- public List<RequestMetadata> getRequests() {
- return this.dqpCore.getRequests();
- }
-
- @Override
- public List<RequestMetadata> getLongRunningRequests() {
- return this.dqpCore.getLongRunningRequests();
- }
-
- @Override
- public WorkerPoolStatisticsMetadata getWorkerPoolStatistics(){
- return this.dqpCore.getWorkerPoolStatistics();
- }
-
- @Override
- public void terminateSession(String terminateeId) {
- this.sessionService.terminateSession(terminateeId, DQPWorkContext.getWorkContext().getSessionId());
- }
-
- @Override
- public boolean cancelRequest(String sessionId, long executionId) throws AdminException {
- try {
- return this.dqpCore.cancelRequest(sessionId, executionId);
- } catch (TeiidComponentException e) {
- throw new AdminComponentException(e);
- }
- }
-
- @Override
- public Collection<SessionMetadata> getActiveSessions() throws AdminException {
- try {
- return this.sessionService.getActiveSessions();
- } catch (SessionServiceException e) {
- throw new AdminComponentException(e);
- }
- }
-
- @Override
- public int getActiveSessionsCount() throws AdminException{
- try {
- return this.sessionService.getActiveSessionsCount();
- } catch (SessionServiceException e) {
- throw new AdminComponentException(e);
- }
- }
-
- @Override
- public Collection<TransactionMetadata> getTransactions() {
- return this.dqpCore.getTransactions();
- }
-
- @Override
- public void terminateTransaction(String xid) throws AdminException {
- this.dqpCore.terminateTransaction(xid);
- }
-
- @Override
- public List<List> executeQuery(final String vdbName, final int version, final String command, final long timoutInMilli) throws AdminException {
-
- String user = "JOPR ADMIN"; //$NON-NLS-1$
- LogManager.logDetail(LogConstants.CTX_RUNTIME, IntegrationPlugin.Util.getString("admin_executing", user, command)); //$NON-NLS-1$
-
- SessionMetadata session = createTemporarySession(vdbName, version, user);
-
- final long requestID = 0L;
-
- DQPWorkContext context = new DQPWorkContext();
- context.setSession(session);
-
- try {
- return context.runInContext(new Callable<List<List>>() {
- @Override
- public List<List> call() throws Exception {
- ArrayList<List> results = new ArrayList<List>();
-
- long start = System.currentTimeMillis();
- RequestMessage request = new RequestMessage(command);
- request.setExecutionId(0L);
- request.setRowLimit(getMaxRowsFetchSize()); // this would limit the number of rows that are returned.
- Future<ResultsMessage> message = dqpCore.executeRequest(requestID, request);
- ResultsMessage rm = null;
- if (timoutInMilli < 0) {
- rm = message.get();
- } else {
- rm = message.get(timoutInMilli, TimeUnit.MILLISECONDS);
- }
- if (rm.getException() != null) {
- throw new AdminProcessingException(rm.getException());
- }
-
- if (rm.isUpdateResult()) {
- results.addAll(new ArrayList(Arrays.asList("update count"))); //$NON-NLS-1$
- results.addAll(Arrays.asList(rm.getResults()));
- }
- else {
- results.addAll(new ArrayList(Arrays.asList(rm.getColumnNames())));
- results.addAll(Arrays.asList(fixResults(rm.getResults())));
-
- while (rm.getFinalRow() == -1 || rm.getLastRow() < rm.getFinalRow()) {
- long elapsed = System.currentTimeMillis() - start;
- message = dqpCore.processCursorRequest(requestID, rm.getLastRow()+1, 1024);
- rm = message.get(timoutInMilli-elapsed, TimeUnit.MILLISECONDS);
- results.addAll(Arrays.asList(fixResults(rm.getResults())));
- }
- }
-
- long elapsed = System.currentTimeMillis() - start;
- ResultsFuture<?> response = dqpCore.closeRequest(requestID);
- response.get(timoutInMilli-elapsed, TimeUnit.MILLISECONDS);
- return results;
- }
- });
- } catch (Throwable t) {
- throw new AdminProcessingException(t);
- } finally {
- try {
- sessionService.closeSession(session.getSessionId());
- } catch (InvalidSessionException e) { //ignore
- }
- }
- }
-
- private SessionMetadata createTemporarySession(final String vdbName, final int version, final String user)
- throws AdminProcessingException {
- Properties properties = new Properties();
- properties.setProperty(TeiidURL.JDBC.VDB_NAME, vdbName);
- properties.setProperty(TeiidURL.JDBC.VDB_VERSION, String.valueOf(version));
-
- SessionMetadata session = null;
- try {
- session = this.sessionService.createSession(user, null, "JOPR", properties, false); //$NON-NLS-1$
- } catch (SessionServiceException e1) {
- throw new AdminProcessingException(e1);
- } catch (LoginException e1) {
- throw new AdminProcessingException(e1);
- }
- return session;
- }
-
- /**
- * Managed Object framework has bug that does not currently allow
- * sending a NULL in the Collection Value, so sending literal string "null".
- * If you send them as Array Value, the MO is packaged as composite object and would like
- * all the elements in array to be same type which is not the case with results.
- */
- List[] fixResults(List[] rows) throws SQLException {
- List[] newResults = new List[rows.length];
-
- for(int i = 0; i < rows.length; i++) {
- List row = rows[i];
- ArrayList<Object> newRow = new ArrayList<Object>();
- for (Object col:row) {
- if (col == null) {
- newRow.add("null"); //$NON-NLS-1$
- }
- else {
- if (col instanceof Number || col instanceof String || col instanceof Character) {
- newRow.add(col);
- }
- else if (col instanceof Blob) {
- newRow.add("blob"); //$NON-NLS-1$
- }
- else if (col instanceof Clob) {
- newRow.add("clob"); //$NON-NLS-1$
- }
- else if (col instanceof SQLXML) {
- SQLXML xml = (SQLXML)col;
- newRow.add(xml.getString());
- }
- else {
- newRow.add(col.toString());
- }
- }
- }
- newResults[i] = newRow;
- }
- return newResults;
- }
-
- @Override
public void updateMatViewRow(String vdbName, int vdbVersion, String schema,
String viewName, List<?> tuple, boolean delete) {
- VDBMetaData metadata = this.vdbRepository.getVDB(vdbName, vdbVersion);
+ VDBMetaData metadata = getVdbRepository().getVDB(vdbName, vdbVersion);
if (metadata != null) {
GlobalTableStore gts = metadata.getAttachment(GlobalTableStore.class);
if (gts != null) {
@@ -565,8 +178,7 @@
}
@Override
- public void dataModification(String vdbName, int vdbVersion, String schema,
- String... tableNames) {
+ public void dataModification(String vdbName, int vdbVersion, String schema, String... tableNames) {
updateModified(true, vdbName, vdbVersion, schema, tableNames);
}
@@ -628,7 +240,7 @@
}
private Schema getSchema(String vdbName, int vdbVersion, String schemaName) {
- VDBMetaData vdb = this.vdbRepository.getVDB(vdbName, vdbVersion);
+ VDBMetaData vdb = getVdbRepository().getVDB(vdbName, vdbVersion);
if (vdb == null) {
return null;
}
@@ -647,12 +259,11 @@
if (t == null) {
return;
}
- DdlPlan.alterInsteadOfTrigger(this.vdbRepository.getVDB(vdbName, vdbVersion), t, triggerDefinition, enabled, triggerEvent);
+ DdlPlan.alterInsteadOfTrigger(getVdbRepository().getVDB(vdbName, vdbVersion), t, triggerDefinition, enabled, triggerEvent);
}
@Override
- public void setProcedureDefinition(String vdbName, int vdbVersion,
- String schema, String procName, String definition) {
+ public void setProcedureDefinition(String vdbName, int vdbVersion,String schema, String procName, String definition) {
Schema s = getSchema(vdbName, vdbVersion, schema);
if (s == null) {
return;
@@ -661,23 +272,22 @@
if (p == null) {
return;
}
- DdlPlan.alterProcedureDefinition(this.vdbRepository.getVDB(vdbName, vdbVersion), p, definition);
+ DdlPlan.alterProcedureDefinition(getVdbRepository().getVDB(vdbName, vdbVersion), p, definition);
}
@Override
- public void setViewDefinition(String vdbName, int vdbVersion,
- String schema, String viewName, String definition) {
+ public void setViewDefinition(String vdbName, int vdbVersion, String schema, String viewName, String definition) {
Table t = getTable(vdbName, vdbVersion, schema, viewName);
if (t == null) {
return;
}
- DdlPlan.alterView(this.vdbRepository.getVDB(vdbName, vdbVersion), t, definition);
+ DdlPlan.alterView(getVdbRepository().getVDB(vdbName, vdbVersion), t, definition);
}
@Override
public void setProperty(String vdbName, int vdbVersion, String uuid,
String name, String value) {
- VDBMetaData vdb = this.vdbRepository.getVDB(vdbName, vdbVersion);
+ VDBMetaData vdb = getVdbRepository().getVDB(vdbName, vdbVersion);
if (vdb == null) {
return;
}
@@ -696,22 +306,6 @@
return this.eventDistributorProxy;
}
- public String getRuntimeVersion() {
- return ApplicationInfo.getInstance().getBuildNumber();
- }
-
- public void setSessionMaxLimit(long limit) {
- this.sessionMaxLimit = limit;
- }
-
- public void setSessionExpirationTimeLimit(long limit) {
- this.sessionExpirationTimeLimit = limit;
- }
-
- public void addSecurityDomain(String domain) {
- this.securityDomainNames.add(domain);
- }
-
public InjectedValue<SessionAwareCache> getResultSetCacheInjector() {
return resultSetCacheInjector;
}
@@ -727,6 +321,10 @@
public InjectedValue<VDBRepository> getVdbRepositoryInjector() {
return vdbRepositoryInjector;
}
+
+ private VDBRepository getVdbRepository() {
+ return vdbRepositoryInjector.getValue();
+ }
public InjectedValue<AuthorizationValidator> getAuthorizationValidatorInjector() {
return authorizationValidatorInjector;
@@ -736,10 +334,6 @@
return bufferServiceInjector;
}
- public InjectedValue<SocketBinding> getJdbcSocketBindingInjector() {
- return jdbcSocketBindingInjector;
- }
-
public InjectedValue<TransactionManager> getTxnManagerInjector() {
return txnManagerInjector;
}
@@ -752,10 +346,6 @@
return workManagerInjector;
}
- public InjectedValue<SocketBinding> getOdbcSocketBindingInjector() {
- return odbcSocketBindingInjector;
- }
-
public InjectedValue<ObjectReplicator> getObjectReplicatorInjector() {
return objectReplicatorInjector;
}
Modified: branches/as7/jboss-integration/src/main/resources/org/teiid/jboss/i18n.properties
===================================================================
--- branches/as7/jboss-integration/src/main/resources/org/teiid/jboss/i18n.properties 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/resources/org/teiid/jboss/i18n.properties 2011-10-12 20:56:57 UTC (rev 3548)
@@ -55,26 +55,19 @@
failed_load_module=Failed to load module "{0}"
translator.add=Add Translator
translator.remove=Remove Translator
-
+protocol_not_found=protocol is not defined for the transport configuration.
+security_enabled=Security Enabled: {0}
DQPCore.unable_to_process_event=Unable to process event.
translator.added = Teiid translator "{0}" added.
+wrong_protocol=Wrong type of protocol supplied
+socket_binding_not_defined=Socket binding not specified for transport {0}; only embedded access is granted.
+embedded_enabled=Teiid Embedded transport enabled.
# subsystem description
-susbsystem.add = Add the Teiid Subsystem
-buffer-service.describe=Buffer Manager Configuration
-resultset-cache.describe=Configuration for result set caching. There will be 2 caches with these settings. One cache holds results that are specific to sessions. The other cache holds vdb scoped results and can be replicated
-preparedplan-cache.describe=PreparedPlan Cache Configuration
-distributed-cache-factory.describe=Distributed Cache Configuration
-query-engine.describe=Teiid Query Engine Configuration
-jdbc.describe=Remote JDBC Access Configuration
-admin.describe=Remote Admin Access Configuration
-odbc.describe=ODBC Access Configuration
-authorization-validator-module.describe=Authorization Module; Implementation of org.teiid.dqp.internal.process.AuthorizationValidator class.
-policy-decider-module.describe=Policy Module; Implementation of org.teiid.PolicyDecider class
+teiid.add = Add the Teiid Subsystem
#Query-ENGINE
-jndi-name.describe=JNDI name of the Teiid Query Engine
-async-thread-group.describe=Thread Pool to be used with Asynchronous operations in Teiid
+async-thread-pool.describe=Thread Pool to be used with Asynchronous operations in Teiid
max-threads.describe=Process pool maximum thread count. (default 64)
max-active-plans.describe=Max active plans (default 20). Increase this value on highly concurrent systems - but ensure that the underlying pools can handle the increased load without timeouts.
thread-count-for-source-concurrency.describe=Max source query concurrency per user request (default 0). \
@@ -84,90 +77,91 @@
time-slice-in-millseconds.describe=Query processor time slice, in milliseconds. (default 2000)
max-row-fetch-size.describe=Maximum allowed fetch size, set via JDBC. User requested value ignored above this value. (default 20480)
lob-chunk-size-in-kb.describe=The max lob chunk size in KB transferred each time when processing blobs, clobs (100KB default)
-use-dataroles.describe=Turn on role checking based upon the data roles defined in VDBs. (default true)
query-threshold-in-seconds.describe=Long running query threshold, after which a alert can be generated by tooling if configured
max-source-rows-allowed.describe=Maximum rows allowed from a source query. -1 indicates no limit. (default -1)
exception-on-max-source-rows.describe=Indicates if an exception should be thrown if the specified value for Maximum Source Rows is exceeded; only up to the maximum rows will be consumed. (default true)
max-odbc-lob-size-allowed.describe=Maximum size of lob allowed through ODBC connection in bytes (default 5MB)
event-distributor-name.describe=The JNDI name of the Teiid Event Distributor
detect-change-events.describe=Set to true for the engine to detect local change events. Should be disabled if using external change data capture tools. (default true)
-security-domain.describe=List of domains to be used to login into Teiid
-admin-security-domain.describe=security domain to be used with Admin API (please do not change this, as this should be same as profile service)
-max-sessions-allowed.describe=Maximum number of sessions allowed by the system (default 5000)
-sessions-expiration-timelimit.describe=Max allowed time before the session is terminated by the system, 0 indicates unlimited (default 0)
allow-env-function.describe=Allow the execution of ENV function. (default false)
+policy-decider-module.describe=Policy Module; Implementation of org.teiid.PolicyDecider class
+authorization-validator-module.describe=Authorization Module; Implementation of org.teiid.dqp.internal.process.AuthorizationValidator class.
#buffer-manager
-use-disk.describe=Use disk for buffer management
-processor-batch-size.describe=The max row count of a batch sent internally within the query processor. Should be <= the connectorBatchSize. (default 512)
-connector-batch-size.describe=The max row count of a batch from a connector. Should be even multiple of processorBatchSize. (default 1024)
-max-file-size.describe= Max File size in MB (default 2GB)
-max-buffer-space.describe=Max storage space, in MB, to be used for buffer files (default 50G)
-max-open-files.describe=Max open buffer files (default 64)
-max-processing-kb.describe=The approximate amount of buffer memory in kilobytes allowable for a single processing operation (sort, grouping, etc.) regardless of existing memory commitments. -1 means to automatically calculate a value (default -1)
-max-reserve-kb.describe=The approximate amount of memory in kilobytes allowed to be held by the buffer manager. -1 means to automatically calculate a value (default -1)
+buffer-service-use-disk.describe=Use disk for buffer management
+buffer-service-processor-batch-size.describe=The max row count of a batch sent internally within the query processor. Should be <= the connectorBatchSize. (default 512)
+buffer-service-connector-batch-size.describe=The max row count of a batch from a connector. Should be even multiple of processorBatchSize. (default 1024)
+buffer-service-max-file-size.describe= Max File size in MB (default 2GB)
+buffer-service-max-buffer-space.describe=Max storage space, in MB, to be used for buffer files (default 50G)
+buffer-service-max-open-files.describe=Max open buffer files (default 64)
+buffer-service-max-processing-kb.describe=The approximate amount of buffer memory in kilobytes allowable for a single processing operation (sort, grouping, etc.) regardless of existing memory commitments. -1 means to automatically calculate a value (default -1)
+buffer-service-max-reserve-kb.describe=The approximate amount of memory in kilobytes allowed to be held by the buffer manager. -1 means to automatically calculate a value (default -1)
-#cache-config
-enabled.describe=enabled
-max-entries.describe=Max Entries allowed
-max-age-in-seconds.describe=Max age in seconds
-max-staleness.describe=Max staleness in seconds. Modifications are based upon data updates -1 indicates no max. (default 60 - 1 minute)
-type.describe=Allowed values are LRU, EXPIRATION. \
- Setting this value to LRU will cause cache hint TTL values \
- to be ignored. (default EXPIRATION)
-container-name.describe=Infinispan cache container name
+#prepared-plan
+preparedplan-cache-max-entries.describe=Max Entries allowed
+preparedplan-cache-max-age-in-seconds.describe=Max age in seconds
+preparedplan-cache-max-staleness.describe=Max staleness in seconds. Modifications are based upon data updates -1 indicates no max. (default 60 - 1 minute)
+
+#resultset cache
+resultset-cache-enable.describe= Resultset cache enabled
+resultset-cache-container-name.describe=Infinispan cache container name
resultset-cache-name.describe = Infinispan cache name for resultset
-
-#cache
-resultset-cache.describe=Resultset cache
-preparedplan-cache.describe=Prepared Plan cache
+resultset-cache-max-age-in-seconds.describe=Max age in seconds
+resultset-cache-max-staleness.describe=Max staleness in seconds. Modifications are based upon data updates -1 indicates no max. (default 60 - 1 minute)
#object replicator
-stack.describe=JGroups stack name for cache replication channel
-cluster-name.describe=cache channel cluster name for cache replication
+object-replicator-stack.describe=JGroups stack name for cache replication channel
+object-replicator-cluster-name.describe=cache channel cluster name for cache replication
-#socket config
+#transport
+transport-name.describe=Name of the transport
+protocol.describe=Transport protocol (allowed=teiid, pg)
+socket-binding.describe=Socket binding to be used for the transport
+input-buffer-size.describe=SO_RCVBUF size, 0 indicates that system default should be used (default 0)
+output-buffer-size.describe=SO_SNDBUF size, 0 indicates that system default should be used (default 0)
max-socket-threads.describe=Max number of threads dedicated to initial request processing. \
Zero indicates the system default of max available processors. (default 0) \
Setting this value above the max available processors is not recommended.
-input-buffer-size.describe=SO_RCVBUF size, 0 indicates that system default should be used (default 0)
-output-buffer-size.describe=SO_SNDBUF size, 0 indicates that system default should be used (default 0)
-socket-binding.describe=Socket binding for the profile
-enabled.describe=enabled
-mode.describe=can be one of disabled, login, or enabled \
+
+authentication-security-domain.describe=Security domain to be enforced with the transport
+authentication-max-sessions-allowed.describe=Maximum number of sessions allowed by the system (default 5000)
+authentication-sessions-expiration-timelimit.describe=Max allowed time before the session is terminated by the system, 0 indicates unlimited (default 0)
+
+pg-max-lob-size-in-bytes.describe=Max LOB size in Postgres protocol, as streaming is not supported
+
+ssl-mode.describe=can be one of disabled, login, or enabled \
disabled = no transport or message level security will be used; \
login = only the login traffic will be encrypted at a message level \
using 128 bit AES with an ephemerial DH key exchange. \
No other config values are needed in this mode; \
enabled = traffic will be secured using this configuration,if the client supports SSL
+ssl-authentication-mode.describe=Authentication Mode (1-way, 2-way, anonymous)
+ssl-ssl-protocol.describe=SSL protocol used
+ssl-keymanagement-algorithm.describe=Use key management algorithm
keystore-name.describe=Keystore File Name
keystore-password.describe=Keystore password
keystore-type.describe=Keystore type
-ssl-protocol.describe=SSL protocol used
-keymanagement-algorithm.describe=Use key management algorithm
truststore-name.describe=Truststore Name
truststore-password.describe=Truststore Password
-authentication-mode.describe=Authentication Mode (1-way, 2-way, anonymous)
-ssl.describe=SSL
+
socket-binding.not_defined=Teiid socket binding not defined for JDBC or ODBC port.
-name.describe = Name of the subsystem
-module.describe = Name of the implementing module
-engine.remove = Remove Teiid query engine
-engine.add = Add Teiid query engine
+
translator.describe = Teiid Translators
translator.add = Add Teiid translator
translator.remove = Remove Teiid translator
-teiid-boot.add = Teiid boot services
+module.describe = Name of the module that implements the translator
+name.describe = name of translator
total-entries.describe = Total entries in Cache
hit-ratio.describe = Cache Hit ratio
request-count.describe=Total request count
+transport.describe = transport description
+
engine_service_not_found= Query Engine Service with name {0} is not found
translator.removed = Translator "{0}" removed
translator.failed-to-load = Translator "{0}" not found in the module "{1}"
cache-container-name-required=container-name required for the resultset cache configuration
-object-replicator.describe=Object Replication
enable.describe=enabled
clearing_cache=Clearing cache {0} for all the deployed vdbs.
@@ -192,13 +186,12 @@
sql-query.missing=Parameter "sql-query" is required
timeout-in-milli.missing=Parameter "timeout-in-milli" is required
translator-name.missing=Parameter "translator-name" is required
-data-role.missing = Data Role is missing from parameters
-mapped-role.missing = security domain role is missing from parameters
-source-name.missing=Model source name is required
-ds-name.missing=Data source name required
-translator-name.missing=translator name required
-model-name.missing=Model Name required
-connection-type.missing = VDB connection type is required
+data-role.missing = Parameter "data-role" is required
+mapped-role.missing = Parameter "mapped-role" is required
+source-name.missing=Parameter "source-name" is required
+ds-name.missing=Parameter "ds-name" is required
+model-name.missing=Parameter "model-name" is required
+connection-type.missing = Parameter "connection-type" is required. Must be one of [NONE, BY_VERSION, ANY]
Modified: branches/as7/jboss-integration/src/main/resources/schema/jboss-teiid.xsd
===================================================================
--- branches/as7/jboss-integration/src/main/resources/schema/jboss-teiid.xsd 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/main/resources/schema/jboss-teiid.xsd 2011-10-12 20:56:57 UTC (rev 3548)
@@ -38,136 +38,72 @@
</xs:annotation>
</xs:element>
- <xs:element name="async-thread-group" type="xs:string" minOccurs="0" maxOccurs="1" default="teiid-async">
+ <xs:element name="async-thread-pool" type="xs:string" minOccurs="0" maxOccurs="1">
<xs:annotation>
<xs:documentation>Thread group to use for Async Processing</xs:documentation>
</xs:annotation>
- </xs:element>
- <xs:element name="buffer-service" type="buffer-service-type" maxOccurs="1" minOccurs="1">
+ </xs:element>
+
+ <xs:element name="buffer-service" maxOccurs="1" minOccurs="1">
<xs:annotation>
<xs:documentation>Buffer manager information</xs:documentation>
</xs:annotation>
- </xs:element>
-
- <xs:element name="authorization-validator-module" type="xs:string" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>An authorization validator that by default uses data role information stored in VDBs. Provide module name.</xs:documentation>
- </xs:annotation>
- </xs:element>
-
- <xs:element name="policy-decider-module" type="xs:string" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>A policy decider that uses data role information stored in VDBs, Provide module name.</xs:documentation>
- </xs:annotation>
- </xs:element>
-
- <xs:element name="resultset-cache" maxOccurs="1" minOccurs="0">
- <xs:annotation>
- <xs:documentation>Configuration for result set caching.</xs:documentation>
- </xs:annotation>
<xs:complexType>
- <xs:attribute name="name" type="xs:string" use="required"/>
- <xs:attribute name="container-name" type="xs:string" use="required"/>
- <xs:attribute name="enable" type="xs:boolean" default="true"/>
- <xs:attribute name="max-staleness" type="xs:int" default="60"/>
+ <xs:attribute name="use-disk" type="xs:string">
+ <xs:annotation>
+ <xs:documentation>Use disk for buffer management</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="processor-batch-size" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>The max row count of a batch sent internally within the query processor. Should be <= the connectorBatchSize. (default 512)</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="connector-batch-size" type="xs:int">
+ <xs:annotation>
+ <xs:documentation> The max row count of a batch from a connector. Should be even multiple of processorBatchSize. (default 1024) </xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="max-processing-kb" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>The approximate amount of memory in kilobytes allowed to be held by the buffer manager. -1 means to automatically calculate a value (default -1). See the admin guide for more.</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="max-reserve-kb" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>The approximate amount of buffer memory in kilobytes allowable for a single processing operation (sort, grouping, etc.) regardless of existing memory commitments. -1 means to automatically calculate a value (default -1). See the admin guide for more</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="max-file-size" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>Max File size in MB (default 2GB)</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="max-buffer-space" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>Max storage space, in MB, to be used for buffer files (default 50G) </xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="max-open-files" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>Max open buffer files (default 64)</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
</xs:complexType>
</xs:element>
- <xs:element name="preparedplan-cache" maxOccurs="1" minOccurs="0">
- <xs:annotation>
- <xs:documentation>Configuration for prepared plan caching. (local memory only)</xs:documentation>
- </xs:annotation>
- <xs:complexType>
- <xs:attribute name="max-entries" type="xs:int" default="512"/>
- <xs:attribute name="max-age-in-seconds" type="xs:int" default="28800"/>
- <xs:attribute name="max-staleness" type="xs:int" default="0"/>
- </xs:complexType>
- </xs:element>
-
- <xs:element name="object-replicator" maxOccurs="1" minOccurs="0">
- <xs:annotation>
- <xs:documentation>Cluster wide results distributor </xs:documentation>
- </xs:annotation>
- <xs:complexType>
- <xs:attribute name="stack" type="xs:string" use="required"/>
- <xs:attribute name="cluster-name" type="xs:string" use="required"/>
- </xs:complexType>
- </xs:element>
-
- <xs:element name="query-engine" type="runtime-engine-type" maxOccurs="unbounded" minOccurs="0">
- <xs:annotation>
- <xs:documentation>Main Teiid runtime engine configuration</xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="translator" maxOccurs="unbounded" minOccurs="0">
- <xs:annotation>
- <xs:documentation>Translators defined in the system</xs:documentation>
- </xs:annotation>
- <xs:complexType>
- <xs:attribute name="name" type="xs:string" use="required"/>
- <xs:attribute name="module" type="xs:string" use="required"/>
- </xs:complexType>
- </xs:element>
- </xs:sequence>
- </xs:complexType>
-
- <xs:complexType name="buffer-service-type">
- <xs:sequence>
- <xs:element name="use-disk" type="xs:string" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>Use disk for buffer management</xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="processor-batch-size" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>The max row count of a batch sent internally within the query processor. Should be <= the connectorBatchSize. (default 512)</xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="connector-batch-size" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation> The max row count of a batch from a connector. Should be even multiple of processorBatchSize. (default 1024) </xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="max-processing-kb" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>The approximate amount of memory in kilobytes allowed to be held by the buffer manager. -1 means to automatically calculate a value (default -1). See the admin guide for more.</xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="max-reserve-kb" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>The approximate amount of buffer memory in kilobytes allowable for a single processing operation (sort, grouping, etc.) regardless of existing memory commitments. -1 means to automatically calculate a value (default -1). See the admin guide for more</xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="max-file-size" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>Max File size in MB (default 2GB)</xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="max-buffer-space" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>Max storage space, in MB, to be used for buffer files (default 50G) </xs:documentation>
- </xs:annotation>
- </xs:element>
- <xs:element name="max-open-files" type="xs:int" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>Max open buffer files (default 64)</xs:documentation>
- </xs:annotation>
- </xs:element>
- </xs:sequence>
- </xs:complexType>
-
- <xs:complexType name="runtime-engine-type">
- <xs:sequence>
<xs:element name="max-threads" type="xs:int" minOccurs="0" maxOccurs="1" default="64">
<xs:annotation>
<xs:documentation>Process pool maximum thread count. (default 64)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="max-active-plans" type="xs:int" minOccurs="0" maxOccurs="1" default="20">
<xs:annotation>
<xs:documentation>Max active plans (default 20). Increase this value on highly concurrent systems - but ensure that the underlying pools can handle the increased load without timeouts.</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="thread-count-for-source-concurrency" type="xs:int" minOccurs="0" maxOccurs="1" default="0">
<xs:annotation>
<xs:documentation> Max source query concurrency per user request (default 0).
@@ -176,88 +112,243 @@
Any number greater than 1 limits the maximum number of concurrently executing source requests accordingly.</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="time-slice-in-millseconds" type="xs:int" minOccurs="0" maxOccurs="1" default="2000">
<xs:annotation>
<xs:documentation>Query processor time slice, in milliseconds. (default 2000)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="max-row-fetch-size" type="xs:int" minOccurs="0" maxOccurs="1" default="20480">
<xs:annotation>
<xs:documentation>Maximum allowed fetch size, set via JDBC. User requested value ignored above this value. (default 20480)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="lob-chunk-size-in-kb" type="xs:int" minOccurs="0" maxOccurs="1" default="100">
<xs:annotation>
<xs:documentation>The max lob chunk size in KB transferred each time when processing blobs, clobs (100KB default)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="query-threshold-in-seconds" type="xs:int" minOccurs="0" maxOccurs="1" default="600">
<xs:annotation>
<xs:documentation>Long running query threshold, after which a alert can be generated by tooling if configured (600 secs)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="max-source-rows-allowed" type="xs:int" minOccurs="0" maxOccurs="1" default="-1">
<xs:annotation>
<xs:documentation>Maximum rows allowed from a source query. -1 indicates no limit. (default -1)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="exception-on-max-source-rows" type="xs:boolean" minOccurs="0" maxOccurs="1" default="true">
<xs:annotation>
<xs:documentation>Indicates if an exception should be thrown if the specified value for Maximum Source Rows is exceeded; only up to the maximum rows will be consumed. (default true)</xs:documentation>
</xs:annotation>
</xs:element>
+
<xs:element name="max-odbc-lob-size-allowed" type="xs:int" minOccurs="0" maxOccurs="1" default="5242880">
<xs:annotation>
<xs:documentation>Maximum size of lob allowed through ODBC connection in bytes (default 5MB)</xs:documentation>
</xs:annotation>
</xs:element>
- <xs:element name="event-distributor-name" type="xs:string" minOccurs="0" maxOccurs="1">
- <xs:annotation>
- <xs:documentation>The JNDI name of the Teiid Event Distributor</xs:documentation>
- </xs:annotation>
- </xs:element>
+
<xs:element name="detect-change-events" type="xs:boolean" minOccurs="0" maxOccurs="1" default="true">
<xs:annotation>
<xs:documentation>Set to true for the engine to detect local change events. Should be disabled if using external change data capture tools. (default true)</xs:documentation>
</xs:annotation>
+ </xs:element>
+
+ <xs:element name="authorization-validator" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="module" type="xs:string" use="required">
+ <xs:annotation>
+ <xs:documentation>An authorization validator that by default uses data role information stored in VDBs. Provide module name.</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ </xs:complexType>
</xs:element>
- <xs:element name="security-domain" type="xs:string" minOccurs="0" maxOccurs="unbounded">
+
+ <xs:element name="policy-decider" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="module" type="xs:string" use="required">
+ <xs:annotation>
+ <xs:documentation>A policy decider that uses data role information stored in VDBs, Provide module name.</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ </xs:complexType>
+ </xs:element>
+
+ <xs:element name="resultset-cache" maxOccurs="1" minOccurs="0">
<xs:annotation>
- <xs:documentation>Comma separated list of domains to be used to login into Teiid</xs:documentation>
+ <xs:documentation>Configuration for result set caching.</xs:documentation>
</xs:annotation>
+ <xs:complexType>
+ <xs:attribute name="name" type="xs:string" use="required"/>
+ <xs:attribute name="container-name" type="xs:string" use="required"/>
+ <xs:attribute name="enable" type="xs:boolean" default="true"/>
+ <xs:attribute name="max-staleness" type="xs:int" default="60"/>
+ </xs:complexType>
</xs:element>
- <xs:element name="max-sessions-allowed" type="xs:int" minOccurs="0" maxOccurs="1">
+
+ <xs:element name="preparedplan-cache" maxOccurs="1" minOccurs="0">
<xs:annotation>
- <xs:documentation>Maximum number of sessions allowed by the system (default 5000)</xs:documentation>
+ <xs:documentation>Configuration for prepared plan caching. (local memory only)</xs:documentation>
</xs:annotation>
+ <xs:complexType>
+ <xs:attribute name="max-entries" type="xs:int" default="512"/>
+ <xs:attribute name="max-age-in-seconds" type="xs:int" default="28800"/>
+ <xs:attribute name="max-staleness" type="xs:int" default="0"/>
+ </xs:complexType>
</xs:element>
- <xs:element name="sessions-expiration-timelimit" type="xs:int" minOccurs="0" maxOccurs="1">
+
+ <xs:element name="object-replicator" maxOccurs="1" minOccurs="0">
<xs:annotation>
- <xs:documentation>Max allowed time before the session is terminated by the system, 0 indicates unlimited (default 0)</xs:documentation>
+ <xs:documentation>Cluster wide results distributor </xs:documentation>
</xs:annotation>
- </xs:element>
- <xs:element name="jdbc" type="socket-config" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="stack" type="xs:string" use="required"/>
+ <xs:attribute name="cluster-name" type="xs:string" use="required"/>
+ </xs:complexType>
+ </xs:element>
+
+ <xs:element name="transport" type="transport-type" maxOccurs="unbounded" minOccurs="0">
<xs:annotation>
- <xs:documentation>jdbc port confguration</xs:documentation>
+ <xs:documentation>Teiid transport configuration</xs:documentation>
</xs:annotation>
</xs:element>
- <xs:element name="odbc" type="socket-config" minOccurs="0" maxOccurs="1">
+
+ <xs:element name="translator" maxOccurs="unbounded" minOccurs="0">
<xs:annotation>
- <xs:documentation>odbc port configuration</xs:documentation>
+ <xs:documentation>Translators defined in the system</xs:documentation>
</xs:annotation>
- </xs:element>
+ <xs:complexType>
+ <xs:attribute name="name" type="xs:string" use="required"/>
+ <xs:attribute name="module" type="xs:string" use="required"/>
+ </xs:complexType>
+ </xs:element>
</xs:sequence>
- <xs:attribute name="name" type="xs:string" use="required"/>
</xs:complexType>
- <xs:complexType name="socket-config">
+ <xs:complexType name="transport-type">
<xs:sequence>
- <xs:element name="ssl" type="ssl-config" minOccurs="0" maxOccurs="1">
+ <xs:element name="authentication" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="security-domain" type="xs:string">
+ <xs:annotation>
+ <xs:documentation>Comma separated list of domains to be used to login into Teiid</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="max-sessions-allowed" type="xs:int" default="5000">
+ <xs:annotation>
+ <xs:documentation>Maximum number of sessions allowed by the system (default 5000)</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="sessions-expiration-timelimit" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>Max allowed time before the session is terminated by the system, 0 indicates unlimited (default 0)</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ </xs:complexType>
+ </xs:element>
+
+ <xs:element name="pg" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="max-lob-size-in-bytes" type="xs:int">
+ <xs:annotation>
+ <xs:documentation>Max Lob size allowed in the PG transport</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ </xs:complexType>
+ </xs:element>
+
+
+ <xs:element name="ssl" minOccurs="0" maxOccurs="1">
<xs:annotation>
<xs:documentation>SSL Configuration</xs:documentation>
</xs:annotation>
- </xs:element>
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element name="keystore" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="name" type="xs:string">
+ <xs:annotation>
+ <xs:documentation>Keystore Name</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="password" type="xs:string">
+ <xs:annotation>
+ <xs:documentation>Keystore Password</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="type" type="xs:string" default="JKS">
+ <xs:annotation>
+ <xs:documentation>Keystore Type</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ </xs:complexType>
+ </xs:element>
+ <xs:element name="truststore" minOccurs="0" maxOccurs="1">
+ <xs:complexType>
+ <xs:attribute name="name" type="xs:string">
+ <xs:annotation>
+ <xs:documentation>Truststore Name</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ <xs:attribute name="password" type="xs:string">
+ <xs:annotation>
+ <xs:documentation>Truststore Password</xs:documentation>
+ </xs:annotation>
+ </xs:attribute>
+ </xs:complexType>
+ </xs:element>
+ </xs:sequence>
+ <xs:attribute name="mode" default="disabled">
+ <xs:annotation>
+ <xs:documentation>can be one of disabled, login, or enabled
+ disabled = no transport or message level security will be used
+ login = only the login traffic will be encrypted at a message level
+ using 128 bit AES with an ephemerial DH key exchange.
+ No other config values are needed in this mode
+ enabled = traffic will be secured using this configuration</xs:documentation>
+ </xs:annotation>
+ <xs:simpleType>
+ <xs:restriction base="xs:string">
+ <xs:enumeration value="login" />
+ <xs:enumeration value="enabled" />
+ <xs:enumeration value="disabled" />
+ </xs:restriction>
+ </xs:simpleType>
+ </xs:attribute>
+ <xs:attribute name="authentication-mode" default="anonymous">
+ <xs:annotation>
+ <xs:documentation>1-way, 2-way, anonymous</xs:documentation>
+ </xs:annotation>
+ <xs:simpleType>
+ <xs:restriction base="xs:string">
+ <xs:enumeration value="1-way" />
+ <xs:enumeration value="2-way" />
+ <xs:enumeration value="anonymous" />
+ </xs:restriction>
+ </xs:simpleType>
+ </xs:attribute>
+ <xs:attribute name="ssl-protocol" type="xs:string"/>
+ <xs:attribute name="keymanagement-algorithm" type="xs:string"/>
+
+ </xs:complexType>
+ </xs:element>
</xs:sequence>
+ <xs:attribute name="name" type="xs:string" use="required"/>
+ <xs:attribute name="protocol" default="teiid">
+ <xs:simpleType>
+ <xs:restriction base="xs:string">
+ <xs:enumeration value="teiid" />
+ <xs:enumeration value="pg" />
+ </xs:restriction>
+ </xs:simpleType>
+ </xs:attribute>
<xs:attribute name="socket-binding" type="xs:string" use="required"/>
<xs:attribute name="input-buffer-size" type="xs:int">
<xs:annotation>
@@ -278,45 +369,4 @@
</xs:attribute>
</xs:complexType>
- <xs:complexType name="ssl-config">
- <xs:sequence>
- <xs:element name="mode" minOccurs="1" maxOccurs="1" default="disabled">
- <xs:annotation>
- <xs:documentation>can be one of disabled, login, or enabled
- disabled = no transport or message level security will be used
- login = only the login traffic will be encrypted at a message level
- using 128 bit AES with an ephemerial DH key exchange.
- No other config values are needed in this mode
- enabled = traffic will be secured using this configuration</xs:documentation>
- </xs:annotation>
- <xs:simpleType>
- <xs:restriction base="xs:string">
- <xs:enumeration value="login" />
- <xs:enumeration value="enabled" />
- <xs:enumeration value="disabled" />
- </xs:restriction>
- </xs:simpleType>
- </xs:element>
- <xs:element name="authentication-mode" minOccurs="0" maxOccurs="1" default="anonymous">
- <xs:annotation>
- <xs:documentation>1-way, 2-way, anonymous</xs:documentation>
- </xs:annotation>
- <xs:simpleType>
- <xs:restriction base="xs:string">
- <xs:enumeration value="1-way" />
- <xs:enumeration value="2-way" />
- <xs:enumeration value="anonymous" />
- </xs:restriction>
- </xs:simpleType>
- </xs:element>
- <xs:element name="ssl-protocol" type="xs:string" minOccurs="0" maxOccurs="1" />
- <xs:element name="keymanagement-algorithm" type="xs:string" minOccurs="0" maxOccurs="1" />
- <xs:element name="keystore-name" type="xs:string" minOccurs="0" maxOccurs="1" />
- <xs:element name="keystore-password" type="xs:string" minOccurs="0" maxOccurs="1" />
- <xs:element name="keystore-type" type="xs:string" minOccurs="0" maxOccurs="1" />
- <xs:element name="truststore-name" type="xs:string" minOccurs="0" maxOccurs="1" />
- <xs:element name="truststore-password" type="xs:string" minOccurs="0" maxOccurs="1" />
- </xs:sequence>
- </xs:complexType>
-
</xs:schema>
Deleted: branches/as7/jboss-integration/src/test/java/org/teiid/jboss/MockXMLExtendedWriter.java
===================================================================
--- branches/as7/jboss-integration/src/test/java/org/teiid/jboss/MockXMLExtendedWriter.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/test/java/org/teiid/jboss/MockXMLExtendedWriter.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,227 +0,0 @@
-package org.teiid.jboss;
-
-import java.util.Iterator;
-
-import javax.xml.namespace.NamespaceContext;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamWriter;
-
-import org.jboss.staxmapper.XMLExtendedStreamWriter;
-
-public class MockXMLExtendedWriter implements XMLExtendedStreamWriter {
- private XMLStreamWriter writer;
-
- public MockXMLExtendedWriter(XMLStreamWriter writer) {
- this.writer = writer;
- }
-
- public void writeStartElement(String localName) throws XMLStreamException {
- writer.writeStartElement(localName);
- }
-
- public void writeStartElement(String namespaceURI, String localName)
- throws XMLStreamException {
- writer.writeStartElement(namespaceURI, localName);
- }
-
- public void writeStartElement(String prefix, String localName,
- String namespaceURI) throws XMLStreamException {
- writer.writeStartElement(prefix, localName, namespaceURI);
- }
-
- public void writeEmptyElement(String namespaceURI, String localName)
- throws XMLStreamException {
- writer.writeEmptyElement(namespaceURI, localName);
- }
-
- public void writeEmptyElement(String prefix, String localName,
- String namespaceURI) throws XMLStreamException {
- writer.writeEmptyElement(prefix, localName, namespaceURI);
- }
-
- public void writeEmptyElement(String localName) throws XMLStreamException {
- writer.writeEmptyElement(localName);
- }
-
- public void writeEndElement() throws XMLStreamException {
- writer.writeEndElement();
- }
-
- public void writeEndDocument() throws XMLStreamException {
- writer.writeEndDocument();
- }
-
- public void close() throws XMLStreamException {
- writer.close();
- }
-
- public void flush() throws XMLStreamException {
- writer.flush();
- }
-
- public void writeAttribute(String localName, String value)
- throws XMLStreamException {
- writer.writeAttribute(localName, value);
- }
-
- public void writeAttribute(String prefix, String namespaceURI,
- String localName, String value) throws XMLStreamException {
- writer.writeAttribute(prefix, namespaceURI, localName, value);
- }
-
- public void writeAttribute(String namespaceURI, String localName,
- String value) throws XMLStreamException {
- writer.writeAttribute(namespaceURI, localName, value);
- }
-
- public void writeNamespace(String prefix, String namespaceURI)
- throws XMLStreamException {
- writer.writeNamespace(prefix, namespaceURI);
- }
-
- public void writeDefaultNamespace(String namespaceURI)
- throws XMLStreamException {
- writer.writeDefaultNamespace(namespaceURI);
- }
-
- public void writeComment(String data) throws XMLStreamException {
- writer.writeComment(data);
- }
-
- public void writeProcessingInstruction(String target)
- throws XMLStreamException {
- writer.writeProcessingInstruction(target);
- }
-
- public void writeProcessingInstruction(String target, String data)
- throws XMLStreamException {
- writer.writeProcessingInstruction(target, data);
- }
-
- public void writeCData(String data) throws XMLStreamException {
- writer.writeCData(data);
- }
-
- public void writeDTD(String dtd) throws XMLStreamException {
- writer.writeDTD(dtd);
- }
-
- public void writeEntityRef(String name) throws XMLStreamException {
- writer.writeEntityRef(name);
- }
-
- public void writeStartDocument() throws XMLStreamException {
- writer.writeStartDocument();
- }
-
- public void writeStartDocument(String version) throws XMLStreamException {
- writer.writeStartDocument(version);
- }
-
- public void writeStartDocument(String encoding, String version)
- throws XMLStreamException {
- writer.writeStartDocument(encoding, version);
- }
-
- public void writeCharacters(String text) throws XMLStreamException {
- writer.writeCharacters(text);
- }
-
- public void writeCharacters(char[] text, int start, int len)
- throws XMLStreamException {
- writer.writeCharacters(text, start, len);
- }
-
- public String getPrefix(String uri) throws XMLStreamException {
- return writer.getPrefix(uri);
- }
-
- public void setPrefix(String prefix, String uri) throws XMLStreamException {
- writer.setPrefix(prefix, uri);
- }
-
- public void setDefaultNamespace(String uri) throws XMLStreamException {
- writer.setDefaultNamespace(uri);
- }
-
- public void setNamespaceContext(NamespaceContext context)
- throws XMLStreamException {
- writer.setNamespaceContext(context);
- }
-
- public NamespaceContext getNamespaceContext() {
- return writer.getNamespaceContext();
- }
-
- public Object getProperty(String name) throws IllegalArgumentException {
- return writer.getProperty(name);
- }
-
- @Override
- public void writeAttribute(String localName, String[] values) throws XMLStreamException {
- this.writer.writeAttribute(localName, join(values));
-
- }
-
- private static String join(final String[] values) {
- final StringBuilder b = new StringBuilder();
- for (int i = 0, valuesLength = values.length; i < valuesLength; i++) {
- final String s = values[i];
- if (s != null) {
- if (i > 0) {
- b.append(' ');
- }
- b.append(s);
- }
- }
- return b.toString();
- }
-
- private static String join(final Iterable<String> values) {
- final StringBuilder b = new StringBuilder();
- Iterator<String> iterator = values.iterator();
- while (iterator.hasNext()) {
- final String s = iterator.next();
- if (s != null) {
- b.append(s);
- if (iterator.hasNext()) b.append(' ');
- }
- }
- return b.toString();
- }
-
- @Override
- public void writeAttribute(String prefix, String namespaceURI, String localName, String[] values) throws XMLStreamException {
- this.writer.writeAttribute(prefix, namespaceURI, localName, join(values));
-
- }
-
- @Override
- public void writeAttribute(String namespaceURI, String localName, String[] values) throws XMLStreamException {
- this.writer.writeAttribute(namespaceURI, localName, join(values));
- }
-
- @Override
- public void writeAttribute(String localName, Iterable<String> value)
- throws XMLStreamException {
- this.writer.writeAttribute(localName, join(value));
-
- }
-
- @Override
- public void writeAttribute(String prefix, String namespaceURI,
- String localName, Iterable<String> value) throws XMLStreamException {
- this.writer.writeAttribute(prefix, namespaceURI, localName, join(value));
-
- }
-
- @Override
- public void writeAttribute(String namespaceURI, String localName,
- Iterable<String> value) throws XMLStreamException {
- this.writer.writeAttribute(namespaceURI, localName, join(value));
- }
-
- @Override
- public void setUnspecifiedElementNamespace(String namespace) {
- }
-}
Modified: branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidAdminOperations.java
===================================================================
--- branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidAdminOperations.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidAdminOperations.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -5,6 +5,7 @@
import static org.junit.Assert.fail;
import java.io.ByteArrayInputStream;
+import java.io.FileNotFoundException;
import java.io.FileReader;
import java.io.IOException;
import java.net.URL;
@@ -43,22 +44,24 @@
@Test
public void testDescribeHandler() throws Exception {
- //Parse the subsystem xml and install into the first controller
- String subsystemXml ="<subsystem xmlns=\"urn:jboss:domain:teiid:1.0\">" +
- "<async-thread-group>async</async-thread-group>"+
- " <query-engine name=\"default\">" +
- " </query-engine>" +
- "</subsystem>";
- KernelServices servicesA = super.installInController(subsystemXml);
+ String subsystemXml = ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-sample-config.xml"));
+ KernelServices servicesA = super.installInController(
+ new AdditionalInitialization() {
+ @Override
+ protected Type getType() {
+ return Type.MANAGEMENT;
+ }
+ },
+ subsystemXml);
+
+
//Get the model and the describe operations from the first controller
ModelNode modelA = servicesA.readWholeModel();
String marshalled = servicesA.getPersistedSubsystemXml();
ModelNode describeOp = new ModelNode();
describeOp.get(OP).set(DESCRIBE);
- describeOp.get(OP_ADDR).set(
- PathAddress.pathAddress(
- PathElement.pathElement(SUBSYSTEM, TeiidExtension.TEIID_SUBSYSTEM)).toModelNode());
+ describeOp.get(OP_ADDR).set(PathAddress.pathAddress(PathElement.pathElement(SUBSYSTEM, TeiidExtension.TEIID_SUBSYSTEM)).toModelNode());
List<ModelNode> operations = super.checkResultAndGetContents(servicesA.executeOperation(describeOp)).asList();
@@ -69,6 +72,15 @@
//Make sure the models from the two controllers are identical
super.compare(modelA, modelB);
}
+
+ @Test
+ public void testMinimumConfiguration() throws Exception {
+ String subsystemXml = "<subsystem xmlns=\"urn:jboss:domain:teiid:1.0\">\n" +
+ " <async-thread-pool>teiid-async</async-thread-pool>"+
+ "</subsystem>";
+ KernelServices services = super.installInController(subsystemXml);
+ ModelNode model = services.readWholeModel();
+ }
@Test
public void testOutputPerisitence() throws Exception {
@@ -90,7 +102,7 @@
ModelNode model = services.readWholeModel();
String marshalled = services.getPersistedSubsystemXml();
- System.out.println(marshalled);
+ //System.out.println(marshalled);
Assert.assertEquals(marshalled, triggered);
Assert.assertEquals(normalizeXML(marshalled), normalizeXML(triggered));
@@ -123,6 +135,8 @@
@Test
public void testSchema() throws Exception {
String subsystemXml = ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-sample-config.xml"));
+ validate(subsystemXml);
+
KernelServices services = super.installInController(
new AdditionalInitialization() {
@Override
@@ -135,7 +149,12 @@
ModelNode model = services.readWholeModel();
String marshalled = services.getPersistedSubsystemXml();
+ validate(marshalled);
+ }
+
+ private void validate(String marshalled) throws SAXException, IOException {
URL xsdURL = Thread.currentThread().getContextClassLoader().getResource("schema/jboss-teiid.xsd");
+ System.out.println(marshalled);
SchemaFactory factory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
Schema schema = factory.newSchema(xsdURL);
@@ -162,12 +181,12 @@
});
validator.validate(source);
- }
+ }
@Test
public void testSubSystemDescription() throws IOException {
ModelNode node = new ModelNode();
- QueryEngineAdd.describeQueryEngine(node, ATTRIBUTES, IntegrationPlugin.getResourceBundle(null));
+ TeiidBootServicesAdd.describeTeiid(node, ATTRIBUTES, IntegrationPlugin.getResourceBundle(null));
assertEquals(ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-model-config.txt")), node.toString());
}
@@ -192,20 +211,10 @@
@Test
public void testQueryOperatrions() throws Exception {
- String subsystemXml = ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-sample-config.xml"));
-
- KernelServices services = super.installInController(
- new AdditionalInitialization() {
- @Override
- protected Type getType() {
- return Type.MANAGEMENT;
- }
- },
- subsystemXml);
+ KernelServices services = buildSubsystem();
PathAddress addr = PathAddress.pathAddress(
- PathElement.pathElement(SUBSYSTEM, TeiidExtension.TEIID_SUBSYSTEM),
- PathElement.pathElement("query-engine", "default"));
+ PathElement.pathElement(SUBSYSTEM, TeiidExtension.TEIID_SUBSYSTEM));
ModelNode addOp = new ModelNode();
addOp.get(OP).set("read-operation-names");
addOp.get(OP_ADDR).set(addr.toModelNode());
@@ -214,15 +223,92 @@
Assert.assertEquals(SUCCESS, result.get(OUTCOME).asString());
List<String> opNames = getList(result);
- assertEquals(22, opNames.size());
- String [] ops = {"add", "cancel-request", "execute-query", "list-requests", "list-sessions", "list-transactions",
- "long-running-queries", "read-attribute", "read-children-names", "read-children-resources",
- "read-children-types", "read-operation-description", "read-operation-names", "read-resource",
- "read-resource-description", "remove", "requests-per-session", "requests-per-vdb",
- "terminate-session", "terminate-transaction", "workerpool-statistics", "write-attribute"};
+ assertEquals(36, opNames.size());
+ String[] ops = { "add", "add-anyauthenticated-role", "add-data-role",
+ "assign-datasource", "cache-statistics", "cache-types",
+ "cancel-request", "change-vdb-connection-type", "clear-cache",
+ "describe", "execute-query", "get-translator", "get-vdb",
+ "list-requests", "list-sessions", "list-transactions",
+ "list-translators", "list-vdbs", "long-running-queries",
+ "merge-vdbs", "read-attribute", "read-children-names",
+ "read-children-resources", "read-children-types",
+ "read-operation-description", "read-operation-names",
+ "read-resource", "read-resource-description",
+ "remove-anyauthenticated-role", "remove-data-role",
+ "requests-per-session", "requests-per-vdb",
+ "terminate-session", "terminate-transaction",
+ "workerpool-statistics", "write-attribute" };
assertEquals(Arrays.asList(ops), opNames);
}
+ @Test
+ public void testAddRemoveTransport() throws Exception {
+ KernelServices services = buildSubsystem();
+
+ PathAddress addr = PathAddress.pathAddress(PathElement.pathElement(SUBSYSTEM, TeiidExtension.TEIID_SUBSYSTEM));
+
+ // look at current query engines make sure there are only two from configuration.
+ ModelNode read = new ModelNode();
+ read.get(OP).set("read-children-names");
+ read.get(OP_ADDR).set(addr.toModelNode());
+ read.get(CHILD_TYPE).set("transport");
+
+ ModelNode result = services.executeOperation(read);
+ Assert.assertEquals(SUCCESS, result.get(OUTCOME).asString());
+
+ List<String> opNames = getList(result);
+ assertEquals(2, opNames.size());
+ String [] ops = {"jdbc", "odbc"};
+ assertEquals(Arrays.asList(ops), opNames);
+
+ // add transport
+ ModelNode addOp = new ModelNode();
+ addOp.get(OP).set("add");
+ addOp.get(OP_ADDR).set(addr.toModelNode().add("transport", "newbie")); //$NON-NLS-1$);
+ addOp.get("protocol").set("pg");
+ addOp.get("socket-binding").set("socket");
+ addOp.get("authentication-security-domain").set("teiid-security");
+
+ result = services.executeOperation(addOp);
+ Assert.assertEquals(SUCCESS, result.get(OUTCOME).asString());
+
+ result = services.executeOperation(read);
+ Assert.assertEquals(SUCCESS, result.get(OUTCOME).asString());
+ opNames = getList(result);
+ assertEquals(3, opNames.size());
+ String [] ops2 = {"jdbc", "newbie", "odbc"};
+ assertEquals(Arrays.asList(ops2), opNames);
+
+ // add transport
+ ModelNode remove = new ModelNode();
+ addOp.get(OP).set("remove");
+ addOp.get(OP_ADDR).set(addr.toModelNode().add("transport", "jdbc")); //$NON-NLS-1$);
+ result = services.executeOperation(addOp);
+ Assert.assertEquals(SUCCESS, result.get(OUTCOME).asString());
+
+ result = services.executeOperation(read);
+ Assert.assertEquals(SUCCESS, result.get(OUTCOME).asString());
+ opNames = getList(result);
+ assertEquals(2, opNames.size());
+ String [] ops3 = {"newbie", "odbc"};
+ assertEquals(Arrays.asList(ops3), opNames);
+ }
+
+ private KernelServices buildSubsystem() throws IOException,
+ FileNotFoundException, Exception {
+ String subsystemXml = ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-sample-config.xml"));
+
+ KernelServices services = super.installInController(
+ new AdditionalInitialization() {
+ @Override
+ protected Type getType() {
+ return Type.MANAGEMENT;
+ }
+ },
+ subsystemXml);
+ return services;
+ }
+
private static List<String> getList(ModelNode operationResult) {
if(!operationResult.hasDefined("result"))
return Collections.emptyList();
Deleted: branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidConfiguration.java
===================================================================
--- branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidConfiguration.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/test/java/org/teiid/jboss/TestTeiidConfiguration.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,379 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * See the COPYRIGHT.txt file distributed with this work for information
- * regarding copyright ownership. Some portions may be licensed
- * to Red Hat, Inc. under one or more contributor license agreements.
- *
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- *
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
- * 02110-1301 USA.
- */
-package org.teiid.jboss;
-
-import static junit.framework.Assert.assertEquals;
-import static org.jboss.as.controller.descriptions.ModelDescriptionConstants.*;
-import static org.junit.Assert.fail;
-
-import java.io.*;
-import java.net.URL;
-import java.util.*;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import javax.xml.namespace.QName;
-import javax.xml.stream.XMLInputFactory;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamReader;
-import javax.xml.stream.XMLStreamWriter;
-import javax.xml.transform.Source;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import javax.xml.validation.Validator;
-
-import junit.framework.Assert;
-
-import org.jboss.as.controller.*;
-import org.jboss.as.controller.descriptions.DescriptionProvider;
-import org.jboss.as.controller.descriptions.common.CommonProviders;
-import org.jboss.as.controller.operations.common.Util;
-import org.jboss.as.controller.operations.global.GlobalOperationHandlers;
-import org.jboss.as.controller.persistence.ConfigurationPersistenceException;
-import org.jboss.as.controller.persistence.ConfigurationPersister;
-import org.jboss.as.controller.persistence.SubsystemMarshallingContext;
-import org.jboss.as.controller.registry.ManagementResourceRegistration;
-import org.jboss.as.controller.registry.Resource;
-import org.jboss.dmr.ModelNode;
-import org.jboss.dmr.ModelType;
-import org.jboss.dmr.Property;
-import org.jboss.msc.service.*;
-import org.jboss.staxmapper.XMLElementWriter;
-import org.jboss.staxmapper.XMLMapper;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.teiid.core.util.ObjectConverterUtil;
-import org.xml.sax.ErrorHandler;
-import org.xml.sax.SAXException;
-import org.xml.sax.SAXParseException;
-
-@SuppressWarnings("nls")
-public class TestTeiidConfiguration {
- static final DescriptionProvider NULL_PROVIDER = new DescriptionProvider() {
- @Override
- public ModelNode getModelDescription(final Locale locale) {
- return new ModelNode();
- }
- };
- static ModelNode profileAddress = new ModelNode();
- static {
- profileAddress.add("profile", "test");
- }
-
- public void testValidateSchema() throws Exception {
- InputStream content = Thread.currentThread().getContextClassLoader().getResourceAsStream("teiid-sample-config.xml");
- URL xsdURL = Thread.currentThread().getContextClassLoader().getResource("schema/jboss-teiid.xsd");
-
- SchemaFactory factory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
- Schema schema = factory.newSchema(xsdURL);
-
- Validator validator = schema.newValidator();
- Source source = new StreamSource(content);
- validator.setErrorHandler(new ErrorHandler() {
-
- @Override
- public void warning(SAXParseException exception) throws SAXException {
- fail(exception.getMessage());
- }
-
- @Override
- public void fatalError(SAXParseException exception) throws SAXException {
- fail(exception.getMessage());
- }
-
- @Override
- public void error(SAXParseException exception) throws SAXException {
- fail(exception.getMessage());
- }
- });
-
- validator.validate(source);
-
- }
-
- public void testTeiidConfiguration() throws Exception {
- List<ModelNode> updates = createSubSystem(ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-sample-config.xml")));
- assertEquals(3, updates.size());
- for (ModelNode update : updates) {
- try {
- executeForResult(update);
- } catch (OperationFailedException e) {
- throw new RuntimeException(e.getFailureDescription().toString());
- }
- }
-
- ModelNode subsystem = model.require("profile").require("test").require("subsystem").require("teiid");
- ModelNode engine = subsystem.require("query-engine");
- assertEquals(2, engine.keys().size());
- ModelNode defaultEngine = engine.get("default");
- assertEquals("default", defaultEngine.require("name").asString());
-
- ModelNode alternateEngine = engine.get("alternate");
- assertEquals("alternate", alternateEngine.require("name").asString());
- }
-
- @Test
- public void testSimpleTeiidConfiguration() throws Exception {
- List<ModelNode> updates = createSubSystem("<subsystem xmlns=\"urn:jboss:domain:teiid:1.0\">" +
- "<async-thread-group>async</async-thread-group>"+
- " <query-engine name=\"default\">" +
- " </query-engine>" +
- "</subsystem>");
- assertEquals(2, updates.size());
- for (ModelNode update : updates) {
- try {
- executeForResult(update);
- } catch (OperationFailedException e) {
- throw new RuntimeException(e.getFailureDescription().toString());
- }
- }
-
- ModelNode subsystem = model.require("profile").require("test").require("subsystem").require("teiid");
- }
-
-
- private ModelNode model;
-
- private ServiceContainer container;
- private ModelController controller;
-
- @Before
- public void setupController() throws InterruptedException {
- container = ServiceContainer.Factory.create("test");
- ServiceTarget target = container.subTarget();
- ControlledProcessState processState = new ControlledProcessState(true);
- ModelControllerService svc = new ModelControllerService(container, processState);
- ServiceBuilder<ModelController> builder = target.addService(ServiceName.of("ModelController"), svc);
- builder.install();
- svc.latch.await();
- controller = svc.getValue();
- ModelNode setup = Util.getEmptyOperation("setup", new ModelNode());
- controller.execute(setup, null, null, null);
- processState.setRunning();
- }
-
- @After
- public void shutdownServiceContainer() {
- if (container != null) {
- container.shutdown();
- try {
- container.awaitTermination(5, TimeUnit.SECONDS);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- finally {
- container = null;
- }
- }
- }
-
- static List<ModelNode> createSubSystem(String xmlContent) throws Exception {
-
- final Reader reader = new StringReader(xmlContent);
- XMLStreamReader xmlReader = XMLInputFactory.newInstance().createXMLStreamReader(reader);
-
- XMLMapper xmlMapper = XMLMapper.Factory.create();
- xmlMapper.registerRootElement(new QName(Namespace.CURRENT.getUri(), "subsystem") , new TeiidSubsystemParser());
-
- List<ModelNode> updates = new ArrayList<ModelNode>();
- xmlMapper.parseDocument(updates, xmlReader);
-
- // Process subsystems
- for(final ModelNode update : updates) {
- // Process relative subsystem path address
- final ModelNode subsystemAddress = profileAddress.clone();
- for(final Property path : update.get(OP_ADDR).asPropertyList()) {
- subsystemAddress.add(path.getName(), path.getValue().asString());
- }
- update.get(OP_ADDR).set(subsystemAddress);
- }
- return updates;
- }
-
- static class TestNewExtensionContext implements ExtensionContext {
- final ManagementResourceRegistration testProfileRegistration;
- ManagementResourceRegistration createdRegistration;
-
- TestNewExtensionContext(ManagementResourceRegistration testProfileRegistration) {
- this.testProfileRegistration = testProfileRegistration;
- }
-
- @Override
- public SubsystemRegistration registerSubsystem(final String name) throws IllegalArgumentException {
- return new SubsystemRegistration() {
- @Override
- public ManagementResourceRegistration registerSubsystemModel(final DescriptionProvider descriptionProvider) {
- if (descriptionProvider == null) {
- throw new IllegalArgumentException("descriptionProvider is null");
- }
- createdRegistration = testProfileRegistration.registerSubModel(PathElement.pathElement("subsystem", name), descriptionProvider);
- Assert.assertEquals("teiid", name);
- return createdRegistration;
- }
-
- @Override
- public ManagementResourceRegistration registerDeploymentModel(final DescriptionProvider descriptionProvider) {
- throw new IllegalStateException("Not implemented");
- }
-
- @Override
- public void registerXMLElementWriter(XMLElementWriter<SubsystemMarshallingContext> writer) {
- Assert.assertNotNull(writer);
- }
- };
- }
- }
-
- private static final DescriptionProvider profileDescriptionProvider = new DescriptionProvider() {
-
- @Override
- public ModelNode getModelDescription(Locale locale) {
- ModelNode node = new ModelNode();
- node.get(DESCRIPTION).set("A named set of subsystem configs");
- node.get(ATTRIBUTES, NAME, TYPE).set(ModelType.STRING);
- node.get(ATTRIBUTES, NAME, DESCRIPTION).set("The name of the profile");
- node.get(ATTRIBUTES, NAME, REQUIRED).set(true);
- node.get(ATTRIBUTES, NAME, MIN_LENGTH).set(1);
- node.get(CHILDREN, SUBSYSTEM, DESCRIPTION).set("The subsystems that make up the profile");
- node.get(CHILDREN, SUBSYSTEM, MIN_OCCURS).set(1);
- node.get(CHILDREN, SUBSYSTEM, MODEL_DESCRIPTION);
- return node;
- }
- };
-
- public class ModelControllerService extends AbstractControllerService {
-
- private final CountDownLatch latch = new CountDownLatch(1);
-
- ModelControllerService(final ServiceContainer serviceContainer, final ControlledProcessState processState) {
- super(OperationContext.Type.SERVER, new TestConfigurationPersister(), processState, NULL_PROVIDER, null);
- }
-
- @Override
- public void start(StartContext context) throws StartException {
- super.start(context);
- latch.countDown();
- }
-
- protected void finishBoot() throws ConfigurationPersistenceException {
-// controller.finshBoot();
-// configurationPersister.successfulBoot();
- }
-
- protected void initModel(Resource rootResource, ManagementResourceRegistration rootRegistration) {
- rootRegistration.registerOperationHandler(READ_RESOURCE_OPERATION, GlobalOperationHandlers.READ_RESOURCE, CommonProviders.READ_RESOURCE_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_ATTRIBUTE_OPERATION, GlobalOperationHandlers.READ_ATTRIBUTE, CommonProviders.READ_ATTRIBUTE_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_RESOURCE_DESCRIPTION_OPERATION, GlobalOperationHandlers.READ_RESOURCE_DESCRIPTION, CommonProviders.READ_RESOURCE_DESCRIPTION_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_CHILDREN_NAMES_OPERATION, GlobalOperationHandlers.READ_CHILDREN_NAMES, CommonProviders.READ_CHILDREN_NAMES_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_CHILDREN_TYPES_OPERATION, GlobalOperationHandlers.READ_CHILDREN_TYPES, CommonProviders.READ_CHILDREN_TYPES_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_CHILDREN_RESOURCES_OPERATION, GlobalOperationHandlers.READ_CHILDREN_RESOURCES, CommonProviders.READ_CHILDREN_RESOURCES_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_OPERATION_NAMES_OPERATION, GlobalOperationHandlers.READ_OPERATION_NAMES, CommonProviders.READ_OPERATION_NAMES_PROVIDER, true);
- rootRegistration.registerOperationHandler(READ_OPERATION_DESCRIPTION_OPERATION, GlobalOperationHandlers.READ_OPERATION_DESCRIPTION, CommonProviders.READ_OPERATION_PROVIDER, true);
- rootRegistration.registerOperationHandler(WRITE_ATTRIBUTE_OPERATION, GlobalOperationHandlers.WRITE_ATTRIBUTE, CommonProviders.WRITE_ATTRIBUTE_PROVIDER, true);
-
- rootRegistration.registerOperationHandler("setup", new OperationStepHandler() {
- @Override
- public void execute(OperationContext context, ModelNode operation) throws OperationFailedException {
- context.createResource(PathAddress.EMPTY_ADDRESS.append(PathElement.pathElement("profile", "test")));
- context.completeStep();
- }
- }, new DescriptionProvider() {
- @Override
- public ModelNode getModelDescription(Locale locale) {
- return new ModelNode();
- }
- });
-
- ManagementResourceRegistration profileRegistration = rootRegistration.registerSubModel(PathElement.pathElement("profile"), profileDescriptionProvider);
- TestNewExtensionContext context = new TestNewExtensionContext(profileRegistration);
- TeiidExtension extension = new TeiidExtension();
- extension.initialize(context);
- Assert.assertNotNull(context.createdRegistration);
- }
-
- }
-
- private class TestConfigurationPersister implements ConfigurationPersister{
-
- @Override
- public PersistenceResource store(final ModelNode model, Set<PathAddress> affectedAddresses) throws ConfigurationPersistenceException {
- return new PersistenceResource() {
- @Override
- public void commit() {
- TestTeiidConfiguration.this.model = model;
- }
-
- @Override
- public void rollback() {
- }
- };
- }
-
- @Override
- public void marshallAsXml(ModelNode model, OutputStream output) throws ConfigurationPersistenceException {
- }
-
- @Override
- public List<ModelNode> load() throws ConfigurationPersistenceException {
- return Collections.emptyList();
- }
-
- @Override
- public void successfulBoot() throws ConfigurationPersistenceException {
- }
-
- @Override
- public String snapshot() {
- return null;
- }
-
- @Override
- public SnapshotInfo listSnapshots() {
- return NULL_SNAPSHOT_INFO;
- }
-
- @Override
- public void deleteSnapshot(String name) {
- }
- }
-
- /**
- * Override to get the actual result from the response.
- */
- public ModelNode executeForResult(ModelNode operation) throws OperationFailedException {
- ModelNode rsp = controller.execute(operation, null, null, null);
- if (FAILED.equals(rsp.get(OUTCOME).asString())) {
- throw new OperationFailedException(rsp.get(FAILURE_DESCRIPTION));
- }
- return rsp.get(RESULT);
- }
-
-
- @Test
- public void testSubSystemDescription() throws IOException {
- ModelNode node = new ModelNode();
- QueryEngineAdd.describeQueryEngine(node, ATTRIBUTES, IntegrationPlugin.getResourceBundle(null));
- assertEquals(ObjectConverterUtil.convertToString(new FileReader("src/test/resources/teiid-model-config.txt")), node.toString());
- }
-}
Modified: branches/as7/jboss-integration/src/test/resources/teiid-model-config.txt
===================================================================
--- branches/as7/jboss-integration/src/test/resources/teiid-model-config.txt 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/test/resources/teiid-model-config.txt 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,4 +1,17 @@
{"attributes" => {
+ "allow-env-function" => {
+ "type" => BOOLEAN,
+ "description" => "Allow the execution of ENV function. (default false)",
+ "required" => false,
+ "max-occurs" => 1,
+ "default" => false
+ },
+ "async-thread-pool" => {
+ "type" => STRING,
+ "description" => "Thread Pool to be used with Asynchronous operations in Teiid",
+ "required" => true,
+ "max-occurs" => 1
+ },
"max-threads" => {
"type" => INT,
"description" => "Process pool maximum thread count. (default 64)",
@@ -62,13 +75,6 @@
"max-occurs" => 1,
"default" => true
},
- "max-odbc-lob-size-allowed" => {
- "type" => INT,
- "description" => "Maximum size of lob allowed through ODBC connection in bytes (default 5MB)",
- "required" => false,
- "max-occurs" => 1,
- "default" => 5242880
- },
"detect-change-events" => {
"type" => BOOLEAN,
"description" => "Set to true for the engine to detect local change events. Should be disabled if using external change data capture tools. (default true)",
@@ -76,194 +82,131 @@
"max-occurs" => 1,
"default" => true
},
- "security-domain" => {
- "type" => LIST,
- "description" => "List of domains to be used to login into Teiid",
+ "authorization-validator-module" => {
+ "type" => STRING,
+ "description" => "Authorization Module; Implementation of org.teiid.dqp.internal.process.AuthorizationValidator class.",
"required" => false,
"max-occurs" => 1
},
- "max-sessions-allowed" => {
- "type" => INT,
- "description" => "Maximum number of sessions allowed by the system (default 5000)",
- "required" => false,
- "max-occurs" => 1,
- "default" => 5000
- },
- "sessions-expiration-timelimit" => {
- "type" => INT,
- "description" => "Max allowed time before the session is terminated by the system, 0 indicates unlimited (default 0)",
- "required" => false,
- "max-occurs" => 1,
- "default" => 0
- },
- "jdbc-max-socket-threads" => {
- "type" => INT,
- "description" => "Max number of threads dedicated to initial request processing. Zero indicates the system default of max available processors. (default 0) Setting this value above the max available processors is not recommended.",
- "required" => false,
- "max-occurs" => 1,
- "default" => 0
- },
- "jdbc-input-buffer-size" => {
- "type" => INT,
- "description" => "SO_RCVBUF size, 0 indicates that system default should be used (default 0)",
- "required" => false,
- "max-occurs" => 1,
- "default" => 0
- },
- "jdbc-output-buffer-size" => {
- "type" => INT,
- "description" => "SO_SNDBUF size, 0 indicates that system default should be used (default 0)",
- "required" => false,
- "max-occurs" => 1,
- "default" => 0
- },
- "jdbc-socket-binding" => {
+ "policy-decider-module" => {
"type" => STRING,
- "description" => "Socket binding for the profile",
+ "description" => "Policy Module; Implementation of org.teiid.PolicyDecider class ",
"required" => false,
"max-occurs" => 1
},
- "jdbc-ssl-mode" => {
+ "object-replicator-stack" => {
"type" => STRING,
- "description" => "can be one of disabled, login, or enabled disabled = no transport or message level security will be used; login = only the login traffic will be encrypted at a message level using 128 bit AES with an ephemerial DH key exchange. No other config values are needed in this mode; enabled = traffic will be secured using this configuration,if the client supports SSL",
+ "description" => "JGroups stack name for cache replication channel",
"required" => false,
- "max-occurs" => 1,
- "default" => "login"
- },
- "jdbc-ssl-keystore-name" => {
- "type" => STRING,
- "description" => "Keystore File Name",
- "required" => false,
"max-occurs" => 1
},
- "jdbc-ssl-keystore-password" => {
+ "object-replicator-cluster-name" => {
"type" => STRING,
- "description" => "Keystore password",
+ "description" => "cache channel cluster name for cache replication",
"required" => false,
"max-occurs" => 1
},
- "jdbc-ssl-keystore-type" => {
- "type" => STRING,
- "description" => "Keystore type",
+ "buffer-service-use-disk" => {
+ "type" => BOOLEAN,
+ "description" => "Use disk for buffer management",
"required" => false,
"max-occurs" => 1,
- "default" => "JKS"
+ "default" => true
},
- "jdbc-ssl-ssl-protocol" => {
- "type" => STRING,
- "description" => "SSL protocol used",
+ "buffer-service-processor-batch-size" => {
+ "type" => INT,
+ "description" => "The max row count of a batch sent internally within the query processor. Should be <= the connectorBatchSize. (default 512)",
"required" => false,
"max-occurs" => 1,
- "default" => "SSLv3"
+ "default" => 512
},
- "jdbc-ssl-keymanagement-algorithm" => {
- "type" => STRING,
- "description" => "Use key management algorithm",
+ "buffer-service-connector-batch-size" => {
+ "type" => INT,
+ "description" => "The max row count of a batch from a connector. Should be even multiple of processorBatchSize. (default 1024)",
"required" => false,
- "max-occurs" => 1
+ "max-occurs" => 1,
+ "default" => 1024
},
- "jdbc-ssl-truststore-name" => {
- "type" => STRING,
- "description" => "Truststore Name",
+ "buffer-service-max-processing-kb" => {
+ "type" => INT,
+ "description" => "The approximate amount of buffer memory in kilobytes allowable for a single processing operation (sort, grouping, etc.) regardless of existing memory commitments. -1 means to automatically calculate a value (default -1)",
"required" => false,
- "max-occurs" => 1
+ "max-occurs" => 1,
+ "default" => -1
},
- "jdbc-ssl-truststore-password" => {
- "type" => STRING,
- "description" => "Truststore Password",
+ "buffer-service-max-reserve-kb" => {
+ "type" => INT,
+ "description" => "The approximate amount of memory in kilobytes allowed to be held by the buffer manager. -1 means to automatically calculate a value (default -1)",
"required" => false,
- "max-occurs" => 1
+ "max-occurs" => 1,
+ "default" => -1
},
- "jdbc-ssl-authentication-mode" => {
- "type" => STRING,
- "description" => "Authentication Mode (1-way, 2-way, anonymous)",
+ "buffer-service-max-file-size" => {
+ "type" => LONG,
+ "description" => "Max File size in MB (default 2GB)",
"required" => false,
"max-occurs" => 1,
- "default" => "anonymous"
+ "default" => 2048L
},
- "odbc-max-socket-threads" => {
- "type" => INT,
- "description" => "Max number of threads dedicated to initial request processing. Zero indicates the system default of max available processors. (default 0) Setting this value above the max available processors is not recommended.",
+ "buffer-service-max-buffer-space" => {
+ "type" => LONG,
+ "description" => "Max storage space, in MB, to be used for buffer files (default 50G)",
"required" => false,
"max-occurs" => 1,
- "default" => 0
+ "default" => 51200L
},
- "odbc-input-buffer-size" => {
+ "buffer-service-max-open-files" => {
"type" => INT,
- "description" => "SO_RCVBUF size, 0 indicates that system default should be used (default 0)",
+ "description" => "Max open buffer files (default 64)",
"required" => false,
"max-occurs" => 1,
- "default" => 0
+ "default" => 64
},
- "odbc-output-buffer-size" => {
+ "preparedplan-cache-max-entries" => {
"type" => INT,
- "description" => "SO_SNDBUF size, 0 indicates that system default should be used (default 0)",
+ "description" => "Max Entries allowed",
"required" => false,
"max-occurs" => 1,
- "default" => 0
+ "default" => 512
},
- "odbc-socket-binding" => {
- "type" => STRING,
- "description" => "Socket binding for the profile",
+ "preparedplan-cache-max-age-in-seconds" => {
+ "type" => INT,
+ "description" => "Max age in seconds",
"required" => false,
- "max-occurs" => 1
- },
- "odbc-ssl-mode" => {
- "type" => STRING,
- "description" => "can be one of disabled, login, or enabled disabled = no transport or message level security will be used; login = only the login traffic will be encrypted at a message level using 128 bit AES with an ephemerial DH key exchange. No other config values are needed in this mode; enabled = traffic will be secured using this configuration,if the client supports SSL",
- "required" => false,
"max-occurs" => 1,
- "default" => "login"
+ "default" => 28800
},
- "odbc-ssl-keystore-name" => {
- "type" => STRING,
- "description" => "Keystore File Name",
+ "preparedplan-cache-max-staleness" => {
+ "type" => INT,
+ "description" => "Max staleness in seconds. Modifications are based upon data updates -1 indicates no max. (default 60 - 1 minute)",
"required" => false,
- "max-occurs" => 1
- },
- "odbc-ssl-keystore-password" => {
- "type" => STRING,
- "description" => "Keystore password",
- "required" => false,
- "max-occurs" => 1
- },
- "odbc-ssl-keystore-type" => {
- "type" => STRING,
- "description" => "Keystore type",
- "required" => false,
"max-occurs" => 1,
- "default" => "JKS"
+ "default" => 0
},
- "odbc-ssl-ssl-protocol" => {
+ "resultset-cache-name" => {
"type" => STRING,
- "description" => "SSL protocol used",
+ "description" => "Infinispan cache name for resultset",
"required" => false,
"max-occurs" => 1,
- "default" => "SSLv3"
+ "default" => "resultset"
},
- "odbc-ssl-keymanagement-algorithm" => {
+ "resultset-cache-container-name" => {
"type" => STRING,
- "description" => "Use key management algorithm",
+ "description" => "Infinispan cache container name ",
"required" => false,
"max-occurs" => 1
},
- "odbc-ssl-truststore-name" => {
- "type" => STRING,
- "description" => "Truststore Name",
+ "resultset-cache-max-staleness" => {
+ "type" => INT,
+ "description" => "Max staleness in seconds. Modifications are based upon data updates -1 indicates no max. (default 60 - 1 minute)",
"required" => false,
- "max-occurs" => 1
+ "max-occurs" => 1,
+ "default" => 60
},
- "odbc-ssl-truststore-password" => {
- "type" => STRING,
- "description" => "Truststore Password",
+ "resultset-cache-enable" => {
+ "type" => BOOLEAN,
+ "description" => "Resultset cache enabled ",
"required" => false,
"max-occurs" => 1
- },
- "odbc-ssl-authentication-mode" => {
- "type" => STRING,
- "description" => "Authentication Mode (1-way, 2-way, anonymous)",
- "required" => false,
- "max-occurs" => 1,
- "default" => "anonymous"
}
}}
\ No newline at end of file
Modified: branches/as7/jboss-integration/src/test/resources/teiid-model-json.txt
===================================================================
--- branches/as7/jboss-integration/src/test/resources/teiid-model-json.txt 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/test/resources/teiid-model-json.txt 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,64 +1,64 @@
{"subsystem" : {"teiid" : {
- "allow-env-function" : "false",
- "async-thread-group" : "teiid-async",
- "buffer-service-connector-batch-size" : "1024",
- "buffer-service-max-buffer-space" : "51200",
- "buffer-service-max-file-size" : "2048",
- "buffer-service-max-open-files" : "64",
- "buffer-service-max-processing-kb" : "-1",
- "buffer-service-max-reserve-kb" : "-1",
- "buffer-service-processor-batch-size" : "512",
- "buffer-service-use-disk" : "true",
+ "allow-env-function" : false,
+ "async-thread-pool" : "teiid-async",
+ "buffer-service-connector-batch-size" : 1024,
+ "buffer-service-max-buffer-space" : 51200,
+ "buffer-service-max-file-size" : 2048,
+ "buffer-service-max-open-files" : 64,
+ "buffer-service-max-processing-kb" : -1,
+ "buffer-service-max-reserve-kb" : -1,
+ "buffer-service-processor-batch-size" : 512,
+ "buffer-service-use-disk" : true,
+ "detect-change-events" : true,
+ "exception-on-max-source-rows" : true,
+ "lob-chunk-size-in-kb" : 100,
+ "max-active-plans" : 20,
+ "max-row-fetch-size" : 20480,
+ "max-source-rows-allowed" : -1,
+ "max-threads" : 64,
"object-replicator-cluster-name" : "cluster",
"object-replicator-stack" : "stack",
"preparedplan-cache-max-age-in-seconds" : 28800,
"preparedplan-cache-max-entries" : 512,
"preparedplan-cache-max-staleness" : 90,
+ "query-threshold-in-seconds" : 600,
"resultset-cache-container-name" : "container",
"resultset-cache-enable" : false,
"resultset-cache-max-staleness" : 90,
"resultset-cache-name" : "cache",
- "query-engine" : {
- "alternate" : {},
- "default" : {
- "detect-change-events" : true,
- "exception-on-max-source-rows" : true,
- "jdbc-input-buffer-size" : 2,
- "jdbc-max-socket-threads" : 1,
- "jdbc-output-buffer-size" : 3,
- "jdbc-socket-binding" : "teiid-jdbc",
- "lob-chunk-size-in-kb" : 100,
- "max-active-plans" : 20,
- "max-odbc-lob-size-allowed" : 5242880,
- "max-row-fetch-size" : 20480,
- "max-sessions-allowed" : 5000,
- "max-source-rows-allowed" : -1,
- "max-threads" : 64,
- "odbc-input-buffer-size" : 2,
- "odbc-max-socket-threads" : 1,
- "odbc-output-buffer-size" : 3,
- "odbc-socket-binding" : "teiid-odbc",
- "odbc-ssl-authentication-mode" : "1-way",
- "odbc-ssl-keymanagement-algorithm" : "algo",
- "odbc-ssl-keystore-name" : "keystore-name",
- "odbc-ssl-keystore-password" : "keystore passs",
- "odbc-ssl-keystore-type" : "key store type",
- "odbc-ssl-mode" : "login",
- "odbc-ssl-ssl-protocol" : "ssl protocol",
- "odbc-ssl-truststore-name" : "truststore name",
- "odbc-ssl-truststore-password" : "trust pass",
- "query-threshold-in-seconds" : 600,
- "security-domain" : [
- "teiid-security",
- "teiid-security2"
- ],
- "sessions-expiration-timelimit" : 0,
- "thread-count-for-source-concurrency" : 0,
- "time-slice-in-millseconds" : 2000
- }
- },
+ "thread-count-for-source-concurrency" : 0,
+ "time-slice-in-millseconds" : 2000,
"translator" : {
"jdbc-simple" : {"module" : "org.jboss.teiid.translator.jdbc"},
"jdbc-ansi" : {"module" : "org.jboss.teiid.translator.jdbc"}
+ },
+ "transport" : {
+ "jdbc" : {
+ "input-buffer-size" : 2,
+ "max-socket-threads" : 1,
+ "output-buffer-size" : 3,
+ "protocol" : "teiid",
+ "socket-binding" : "teiid-jdbc"
+ },
+ "odbc" : {
+ "authentication-security-domain" : "teiid-security,teiid-security2",
+ "authentication-sessions-expiration-timelimit" : 0,
+ "authentication-max-sessions-allowed" : 5000,
+ "input-buffer-size" : 2,
+ "keystore-name" : "keystore name",
+ "keystore-password" : "keystore passs",
+ "keystore-type" : "keystore type",
+ "max-socket-threads" : 1,
+ "output-buffer-size" : 3,
+ "pg-max-lob-size-in-bytes" : 5242880,
+ "protocol" : "pg",
+ "socket-binding" : "teiid-odbc",
+ "ssl-authentication-mode" : "1-way",
+ "ssl-keymanagement-algorithm" : "algo",
+ "ssl-mode" : "login",
+ "ssl-ssl-protocol" : "ssl-protocol",
+ "truststore-name" : "truststore name",
+ "truststore-password" : "truststore pass"
+ }
}
}}}
\ No newline at end of file
Modified: branches/as7/jboss-integration/src/test/resources/teiid-sample-config.xml
===================================================================
--- branches/as7/jboss-integration/src/test/resources/teiid-sample-config.xml 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/jboss-integration/src/test/resources/teiid-sample-config.xml 2011-10-12 20:56:57 UTC (rev 3548)
@@ -1,64 +1,43 @@
<subsystem xmlns="urn:jboss:domain:teiid:1.0">
- <allow-env-function>false</allow-env-function>
- <async-thread-group>teiid-async</async-thread-group>
+ <allow-env-function>false</allow-env-function>
+ <async-thread-pool>teiid-async</async-thread-pool>
- <buffer-service>
- <use-disk>true</use-disk>
- <processor-batch-size>512</processor-batch-size>
- <connector-batch-size>1024</connector-batch-size>
- <max-processing-kb>-1</max-processing-kb>
- <max-reserve-kb>-1</max-reserve-kb>
- <max-file-size>2048</max-file-size>
- <max-buffer-space>51200</max-buffer-space>
- <max-open-files>64</max-open-files>
- </buffer-service>
- <!--
- <authorization-validator-module>javax.api</authorization-validator-module>
- <policy-decider-module>javax.api</policy-decider-module>
- -->
-
- <resultset-cache name="cache" container-name="container" enable="false" max-staleness="90"/>
-
- <preparedplan-cache max-entries="512" max-age-in-seconds="28800" max-staleness="90"/>
-
- <object-replicator stack="stack" cluster-name="cluster"/>
-
- <query-engine name="default">
- <max-threads>64</max-threads>
- <max-active-plans>20</max-active-plans>
- <thread-count-for-source-concurrency>0</thread-count-for-source-concurrency>
- <time-slice-in-millseconds>2000</time-slice-in-millseconds>
- <max-row-fetch-size>20480</max-row-fetch-size>
- <lob-chunk-size-in-kb>100</lob-chunk-size-in-kb>
- <query-threshold-in-seconds>600</query-threshold-in-seconds>
- <max-source-rows-allowed>-1</max-source-rows-allowed>
- <exception-on-max-source-rows>true</exception-on-max-source-rows>
- <max-odbc-lob-size-allowed>5242880</max-odbc-lob-size-allowed>
- <detect-change-events>true</detect-change-events>
- <security-domain>teiid-security</security-domain>
- <security-domain>teiid-security2</security-domain>
- <max-sessions-allowed>5000</max-sessions-allowed>
- <sessions-expiration-timelimit>0</sessions-expiration-timelimit>
-
- <jdbc socket-binding="teiid-jdbc" max-socket-threads="1" input-buffer-size="2" output-buffer-size="3"/>
+ <buffer-service use-disk="true" processor-batch-size="512" connector-batch-size="1024"
+ max-processing-kb="-1" max-reserve-kb="-1" max-file-size="2048" max-buffer-space="51200" max-open-files="64"/>
- <odbc socket-binding="teiid-odbc" max-socket-threads="1" input-buffer-size="2" output-buffer-size="3">
- <ssl>
- <mode>login</mode>
- <keystore-name>keystore-name</keystore-name>
- <keystore-password>keystore passs</keystore-password>
- <keystore-type>key store type</keystore-type>
- <ssl-protocol>ssl protocol</ssl-protocol>
- <keymanagement-algorithm>algo</keymanagement-algorithm>
- <truststore-name>truststore name</truststore-name>
- <truststore-password>trust pass</truststore-password>
- <authentication-mode>1-way</authentication-mode>
- </ssl>
- </odbc>
- </query-engine>
+ <max-threads>64</max-threads>
+ <max-active-plans>20</max-active-plans>
+ <thread-count-for-source-concurrency>0</thread-count-for-source-concurrency>
+ <time-slice-in-millseconds>2000</time-slice-in-millseconds>
+ <max-row-fetch-size>20480</max-row-fetch-size>
+ <lob-chunk-size-in-kb>100</lob-chunk-size-in-kb>
+ <query-threshold-in-seconds>600</query-threshold-in-seconds>
+ <max-source-rows-allowed>-1</max-source-rows-allowed>
+ <exception-on-max-source-rows>true</exception-on-max-source-rows>
+ <detect-change-events>true</detect-change-events>
- <query-engine name="alternate"/>
+ <!--
+ <authorization-validator module="javax.api"/>
+ <policy-decider module="javax.api"/>
+ -->
+
+ <resultset-cache name="cache" container-name="container" enable="false" max-staleness="90"/>
+
+ <preparedplan-cache max-entries="512" max-age-in-seconds="28800" max-staleness="90"/>
+
+ <object-replicator stack="stack" cluster-name="cluster"/>
+
+ <transport name="jdbc" protocol="teiid" socket-binding="teiid-jdbc" max-socket-threads="1" input-buffer-size="2" output-buffer-size="3"/>
+ <transport name="odbc" protocol="pg" socket-binding="teiid-odbc" max-socket-threads="1" input-buffer-size="2" output-buffer-size="3">
+ <authentication security-domain="teiid-security,teiid-security2" max-sessions-allowed="5000" sessions-expiration-timelimit="0"/>
+ <pg max-lob-size-in-bytes="5242880"/>
+ <ssl mode="login" authentication-mode="1-way" ssl-protocol="ssl-protocol" keymanagement-algorithm="algo">
+ <keystore name="keystore name" password="keystore passs" type="keystore type"/>
+ <truststore name="truststore name" password="truststore pass"/>
+ </ssl>
+ </transport>
+
<translator name="jdbc-simple" module="org.jboss.teiid.translator.jdbc"/>
<translator name="jdbc-ansi" module="org.jboss.teiid.translator.jdbc"/>
</subsystem>
\ No newline at end of file
Modified: branches/as7/runtime/src/main/java/org/teiid/transport/LocalServerConnection.java
===================================================================
--- branches/as7/runtime/src/main/java/org/teiid/transport/LocalServerConnection.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/runtime/src/main/java/org/teiid/transport/LocalServerConnection.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -49,7 +49,7 @@
public class LocalServerConnection implements ServerConnection {
- public static final String TEIID_RUNTIME_CONTEXT = "teiid/queryengine/"; //$NON-NLS-1$
+ public static final String TEIID_RUNTIME_CONTEXT = "teiid/queryengine"; //$NON-NLS-1$
private LogonResult result;
private boolean shutdown;
@@ -69,9 +69,8 @@
protected ClientServiceRegistry getClientServiceRegistry() {
try {
- String engineName = this.connectionProperties.getProperty(TeiidURL.CONNECTION.ENGINE_NAME, "default"); //$NON-NLS-1$
InitialContext ic = new InitialContext();
- return (ClientServiceRegistry)ic.lookup(TEIID_RUNTIME_CONTEXT+engineName);
+ return (ClientServiceRegistry)ic.lookup(TEIID_RUNTIME_CONTEXT);
} catch (NamingException e) {
throw new TeiidRuntimeException(e);
}
Modified: branches/as7/runtime/src/main/java/org/teiid/transport/ODBCSocketListener.java
===================================================================
--- branches/as7/runtime/src/main/java/org/teiid/transport/ODBCSocketListener.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/runtime/src/main/java/org/teiid/transport/ODBCSocketListener.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -42,9 +42,9 @@
private TeiidDriver driver;
private ILogon logonService;
- public ODBCSocketListener(InetSocketAddress address, SocketConfiguration config, StorageManager storageManager, int maxLobSize, ILogon logon) {
+ public ODBCSocketListener(InetSocketAddress address, SocketConfiguration config, final ClientServiceRegistryImpl csr, StorageManager storageManager, int maxLobSize, ILogon logon) {
//the clientserviceregistry isn't actually used by ODBC
- super(address, config, new ClientServiceRegistryImpl(ClientServiceRegistry.Type.ODBC), storageManager);
+ super(address, config, csr, storageManager);
this.maxLobSize = maxLobSize;
this.driver = new TeiidDriver();
this.driver.setEmbeddedProfile(new EmbeddedProfile() {
@@ -52,7 +52,11 @@
protected ServerConnection createServerConnection(Properties info)
throws TeiidException {
//When using the non-blocking api, we don't want to use the calling thread
- return new LocalServerConnection(info, false);
+ return new LocalServerConnection(info, false) {
+ protected ClientServiceRegistry getClientServiceRegistry() {
+ return csr;
+ }
+ };
}
});
this.logonService = logon;
Modified: branches/as7/runtime/src/main/java/org/teiid/transport/SocketConfiguration.java
===================================================================
--- branches/as7/runtime/src/main/java/org/teiid/transport/SocketConfiguration.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/runtime/src/main/java/org/teiid/transport/SocketConfiguration.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -37,7 +37,7 @@
private SSLConfiguration sslConfiguration;
private String hostName;
private String name;
- private String socketBinding;
+ private String protocol = "teiid"; //$NON-NLS-1$
public String getName() {
return name;
@@ -47,14 +47,6 @@
this.name = name;
}
- public String getSocketBinding() {
- return socketBinding;
- }
-
- public void setSocketBinding(String socketBinding) {
- this.socketBinding = socketBinding;
- }
-
public void setBindAddress(String addr) {
this.hostName = addr;
}
@@ -141,4 +133,12 @@
public boolean getSslEnabled() {
return this.sslConfiguration != null && this.sslConfiguration.isSslEnabled();
}
+
+ public String getProtocol() {
+ return protocol;
+ }
+
+ public void setProtocol(String protocol) {
+ this.protocol = protocol;
+ }
}
Modified: branches/as7/test-integration/common/src/test/java/org/teiid/transport/TestODBCSocketTransport.java
===================================================================
--- branches/as7/test-integration/common/src/test/java/org/teiid/transport/TestODBCSocketTransport.java 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/test-integration/common/src/test/java/org/teiid/transport/TestODBCSocketTransport.java 2011-10-12 20:56:57 UTC (rev 3548)
@@ -129,7 +129,7 @@
addr = new InetSocketAddress(0);
config.setBindAddress(addr.getHostName());
config.setPortNumber(0);
- odbcTransport = new ODBCSocketListener(addr, config, BufferManagerFactory.getStandaloneBufferManager(), 100000, Mockito.mock(ILogon.class));
+ odbcTransport = new ODBCSocketListener(addr, config, Mockito.mock(ClientServiceRegistryImpl.class), BufferManagerFactory.getStandaloneBufferManager(), 100000, Mockito.mock(ILogon.class));
odbcTransport.setMaxBufferSize(1000); //set to a small size to ensure buffering over the limit works
FakeServer server = new FakeServer();
server.setUseCallingThread(false);
Modified: branches/as7/test-integration/db/pom.xml
===================================================================
--- branches/as7/test-integration/db/pom.xml 2011-10-09 03:25:35 UTC (rev 3547)
+++ branches/as7/test-integration/db/pom.xml 2011-10-12 20:56:57 UTC (rev 3548)
@@ -78,6 +78,11 @@
<artifactId>ant</artifactId>
<version>1.7.0</version>
</dependency>
+
+ <dependency>
+ <groupId>org.jboss.logging</groupId>
+ <artifactId>jboss-logging</artifactId>
+ </dependency>
<dependency>
<groupId>org.apache.ant</groupId>
13 years, 2 months
teiid SVN: r3547 - trunk/engine/src/main/java/org/teiid/common/buffer/impl.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-08 23:25:35 -0400 (Sat, 08 Oct 2011)
New Revision: 3547
Modified:
trunk/engine/src/main/java/org/teiid/common/buffer/impl/FileStoreCache.java
Log:
TEIID-1750 correcting concurrency issues
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/FileStoreCache.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/FileStoreCache.java 2011-10-09 03:23:46 UTC (rev 3546)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/FileStoreCache.java 2011-10-09 03:25:35 UTC (rev 3547)
@@ -66,6 +66,7 @@
public class FileStoreCache implements Cache {
private static class CacheGroup {
+ private static final int RECLAIM_TAIL_SIZE = IO_BUFFER_SIZE << 5;
private static final int MAX_FREE_SPACE = 1 << 11;
FileStore store;
long tail;
@@ -83,7 +84,7 @@
if (info != null) {
if (info[0] + info[1] == tail) {
tail -= info[1];
- if (store.getLength() - tail > IO_BUFFER_SIZE << 5) {
+ if (store.getLength() - tail > RECLAIM_TAIL_SIZE) {
store.setLength(tail);
}
} else {
@@ -228,9 +229,9 @@
}
@Override
- protected int flushDirect() throws IOException {
- group.store.write(offset + bytesWritten, buffer.array(), 0, buf.position());
- return buf.position();
+ protected int flushDirect(int i) throws IOException {
+ group.store.write(offset + bytesWritten, buffer.array(), 0, i);
+ return i;
}
};
ObjectOutputStream oos = new ObjectOutputStream(fsos);
13 years, 2 months
teiid SVN: r3546 - in trunk: engine/src/main/java/org/teiid/common/buffer and 1 other directories.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-08 23:23:46 -0400 (Sat, 08 Oct 2011)
New Revision: 3546
Modified:
trunk/client/src/main/java/org/teiid/client/ResizingArrayList.java
trunk/engine/src/main/java/org/teiid/common/buffer/LightWeightCopyOnWriteList.java
trunk/engine/src/main/java/org/teiid/common/buffer/SPage.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/OrderedCache.java
Log:
TEIID-1750 correcting concurrency issues
Modified: trunk/client/src/main/java/org/teiid/client/ResizingArrayList.java
===================================================================
--- trunk/client/src/main/java/org/teiid/client/ResizingArrayList.java 2011-10-07 20:31:32 UTC (rev 3545)
+++ trunk/client/src/main/java/org/teiid/client/ResizingArrayList.java 2011-10-09 03:23:46 UTC (rev 3546)
@@ -63,8 +63,13 @@
return (T) elementData[index];
}
+ public int getModCount() {
+ return modCount;
+ }
+
public void add(int index, T element) {
rangeCheck(index, true);
+ modCount++;
ensureCapacity(size+1);
System.arraycopy(elementData, index, elementData, index + 1,
size - index);
@@ -99,6 +104,7 @@
@Override
public boolean addAll(int index, Collection<? extends T> c) {
rangeCheck(index, true);
+ modCount++;
int numNew = c.size();
ensureCapacity(size + numNew);
for (T t : c) {
@@ -111,6 +117,7 @@
@Override
public T remove(int index) {
T oldValue = get(index);
+ modCount++;
int numMoved = size - index - 1;
if (numMoved > 0) {
System.arraycopy(elementData, index+1, elementData, index, numMoved);
@@ -134,6 +141,7 @@
@Override
public void clear() {
+ modCount++;
if (size <= MIN_SHRINK_SIZE) {
for (int i = 0; i < size; i++) {
elementData[i] = null;
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/LightWeightCopyOnWriteList.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/LightWeightCopyOnWriteList.java 2011-10-07 20:31:32 UTC (rev 3545)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/LightWeightCopyOnWriteList.java 2011-10-09 03:23:46 UTC (rev 3546)
@@ -28,6 +28,8 @@
import java.util.List;
import java.util.RandomAccess;
+import org.teiid.client.ResizingArrayList;
+
/**
* Creates a copy of a reference list when modified.
*
@@ -101,7 +103,7 @@
@Override
public void clear() {
if (!modified) {
- list = new ArrayList<T>();
+ list = new ResizingArrayList<T>();
modified = true;
} else {
list.clear();
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/SPage.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/SPage.java 2011-10-07 20:31:32 UTC (rev 3545)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/SPage.java 2011-10-09 03:23:46 UTC (rev 3546)
@@ -25,7 +25,6 @@
import java.lang.ref.PhantomReference;
import java.lang.ref.Reference;
import java.lang.ref.ReferenceQueue;
-import java.util.ArrayList;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Iterator;
@@ -120,10 +119,10 @@
SPage clone = (SPage) super.clone();
clone.stree = tree;
if (children != null) {
- clone.children = new ArrayList<SPage>(children);
+ clone.children = new ResizingArrayList<SPage>(children);
}
if (values != null) {
- clone.values = new ArrayList<List<?>>(values);
+ clone.values = new ResizingArrayList<List<?>>(values);
}
return clone;
} catch (CloneNotSupportedException e) {
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-10-07 20:31:32 UTC (rev 3545)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-10-09 03:23:46 UTC (rev 3546)
@@ -45,6 +45,7 @@
import java.util.concurrent.locks.ReentrantLock;
import org.teiid.client.BatchSerializer;
+import org.teiid.client.ResizingArrayList;
import org.teiid.common.buffer.AutoCleanupUtil;
import org.teiid.common.buffer.BatchManager;
import org.teiid.common.buffer.BufferManager;
@@ -178,8 +179,21 @@
@Override
public void serialize(List<? extends List<?>> obj,
ObjectOutputStream oos) throws IOException {
- //it's expected that the containing structure has updated the lob manager
- BatchSerializer.writeBatch(oos, types, obj);
+ int expectedModCount = 0;
+ ResizingArrayList<?> list = null;
+ if (obj instanceof ResizingArrayList<?>) {
+ list = (ResizingArrayList<?>)obj;
+ }
+ try {
+ //it's expected that the containing structure has updated the lob manager
+ BatchSerializer.writeBatch(oos, types, obj);
+ } catch (IndexOutOfBoundsException e) {
+ //there is a chance of a concurrent persist while modifying
+ //in which case we want to swallow this exception
+ if (list == null || list.getModCount() == expectedModCount) {
+ throw e;
+ }
+ }
}
public int getSizeEstimate(List<? extends List<?>> obj) {
@@ -574,29 +588,34 @@
}
freed += ce.getSizeEstimate();
activeBatchKB.addAndGet(-ce.getSizeEstimate());
- synchronized (ce) {
- if (ce.isPersistent()) {
- continue;
- }
- ce.setPersistent(true);
+ try {
+ evict(ce);
+ } catch (Throwable e) {
+ LogManager.logError(LogConstants.CTX_BUFFER_MGR, e, "Error persisting batch, attempts to read batch "+ ce.getId() +" later will result in an exception"); //$NON-NLS-1$ //$NON-NLS-2$
+ } finally {
+ this.memoryEntries.finishedEviction(ce.getId());
}
- persist(ce);
}
}
- void persist(CacheEntry ce) {
+ void evict(CacheEntry ce) throws Exception {
Serializer<?> s = ce.getSerializer().get();
if (s == null) {
return;
}
- long count = writeCount.incrementAndGet();
- if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
- LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, ce.getId(), "writing batch to storage, total writes: ", count); //$NON-NLS-1$
+ boolean persist = false;
+ synchronized (ce) {
+ if (!ce.isPersistent()) {
+ persist = true;
+ ce.setPersistent(true);
+ }
}
- try {
+ if (persist) {
+ long count = writeCount.incrementAndGet();
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, ce.getId(), "writing batch to storage, total writes: ", count); //$NON-NLS-1$
+ }
cache.add(ce, s);
- } catch (Throwable e) {
- LogManager.logError(LogConstants.CTX_BUFFER_MGR, e, "Error persisting batch, attempts to read batch "+ ce.getId() +" later will result in an exception"); //$NON-NLS-1$ //$NON-NLS-2$
}
if (s.useSoftCache()) {
createSoftReference(ce);
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java 2011-10-07 20:31:32 UTC (rev 3545)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java 2011-10-09 03:23:46 UTC (rev 3546)
@@ -64,14 +64,17 @@
public void flush() throws IOException {
if (buf != null && buf.position() > 0) {
- bytesWritten += flushDirect();
+ bytesWritten += flushDirect(buf.position());
}
buf = null;
}
protected abstract ByteBuffer newBuffer();
- protected abstract int flushDirect() throws IOException;
+ /**
+ * Flush up to i bytes where i is the current position of the buffer
+ */
+ protected abstract int flushDirect(int i) throws IOException;
@Override
public void close() throws IOException {
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/OrderedCache.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/OrderedCache.java 2011-10-07 20:31:32 UTC (rev 3545)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/OrderedCache.java 2011-10-09 03:23:46 UTC (rev 3546)
@@ -31,9 +31,13 @@
protected Map<K, V> map = new ConcurrentHashMap<K, V>();
protected NavigableMap<V, K> expirationQueue = new ConcurrentSkipListMap<V, K>();
+ protected Map<K, V> limbo = new ConcurrentHashMap<K, V>();
public V get(K key) {
V result = map.get(key);
+ if (result == null) {
+ result = limbo.get(key);
+ }
if (result != null) {
synchronized (result) {
expirationQueue.remove(result);
@@ -73,9 +77,14 @@
if (entry == null) {
return null;
}
+ limbo.put(entry.getValue(), entry.getKey());
return map.remove(entry.getValue());
}
+ public void finishedEviction(K key) {
+ limbo.remove(key);
+ }
+
public int size() {
return map.size();
}
13 years, 2 months
teiid SVN: r3545 - in branches/7.4.x/connectors/translator-jdbc/src: test/java/org/teiid/translator/jdbc/ingres and 1 other directory.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-07 16:31:32 -0400 (Fri, 07 Oct 2011)
New Revision: 3545
Modified:
branches/7.4.x/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java
branches/7.4.x/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java
Log:
TEIID-1776 back port of fix
Modified: branches/7.4.x/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java
===================================================================
--- branches/7.4.x/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java 2011-10-07 16:52:06 UTC (rev 3544)
+++ branches/7.4.x/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java 2011-10-07 20:31:32 UTC (rev 3545)
@@ -57,13 +57,14 @@
convert.addTypeMapping("decimal(38,19)", FunctionModifier.BIGDECIMAL); //$NON-NLS-1$
convert.addTypeMapping("decimal(15,0)", FunctionModifier.BIGINTEGER); //$NON-NLS-1$
convert.addTypeMapping("date", FunctionModifier.DATE); //$NON-NLS-1$
- convert.addTypeMapping("time with time zone", FunctionModifier.TIME); //$NON-NLS-1$
- convert.addTypeMapping("timestamp with time zone", FunctionModifier.TIMESTAMP); //$NON-NLS-1$
+ convert.addTypeMapping("time", FunctionModifier.TIME); //$NON-NLS-1$
+ convert.addTypeMapping("timestamp", FunctionModifier.TIMESTAMP); //$NON-NLS-1$
convert.addTypeMapping("char(1)", FunctionModifier.CHAR); //$NON-NLS-1$
convert.addTypeMapping("varchar(4000)", FunctionModifier.STRING); //$NON-NLS-1$
convert.addTypeMapping("blob", FunctionModifier.BLOB); //$NON-NLS-1$
convert.addTypeMapping("clob", FunctionModifier.CLOB); //$NON-NLS-1$
convert.addNumericBooleanConversions();
+ convert.setWideningNumericImplicit(true);
registerFunctionModifier(SourceSystemFunctions.CONVERT, convert);
registerFunctionModifier(SourceSystemFunctions.BITAND, new AliasModifier("bit_and")); //$NON-NLS-1$
Modified: branches/7.4.x/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java
===================================================================
--- branches/7.4.x/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java 2011-10-07 16:52:06 UTC (rev 3544)
+++ branches/7.4.x/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java 2011-10-07 20:31:32 UTC (rev 3545)
@@ -68,7 +68,7 @@
}
@Test public void testBooleanToBigDecimal() throws Exception {
- helpTest(LANG_FACTORY.createLiteral(Boolean.TRUE, Boolean.class), "bigdecimal", "cast(1 AS decimal(38,19))"); //$NON-NLS-1$ //$NON-NLS-2$
+ helpTest(LANG_FACTORY.createLiteral(Boolean.TRUE, Boolean.class), "bigdecimal", "1"); //$NON-NLS-1$ //$NON-NLS-2$
}
// Source = BYTE
@@ -82,7 +82,7 @@
}
@Test public void testBigIntegerToDouble() throws Exception {
- helpTest(LANG_FACTORY.createLiteral(new BigInteger("1"), BigInteger.class), "double", "cast(1 AS float)"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
+ helpTest(LANG_FACTORY.createLiteral(new BigInteger("1"), BigInteger.class), "double", "1"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
}
// Source = FLOAT
@@ -106,7 +106,7 @@
// Source = DATE
@Test public void testDateToTimestamp() throws Exception {
- helpTest(LANG_FACTORY.createLiteral(TimestampUtil.createDate(103, 10, 1), java.sql.Date.class), "timestamp", "cast(DATE '2003-11-01' AS timestamp with time zone)"); //$NON-NLS-1$ //$NON-NLS-2$
+ helpTest(LANG_FACTORY.createLiteral(TimestampUtil.createDate(103, 10, 1), java.sql.Date.class), "timestamp", "cast(DATE '2003-11-01' AS timestamp)"); //$NON-NLS-1$ //$NON-NLS-2$
}
// Source = TIME
13 years, 2 months
teiid SVN: r3544 - in trunk/engine/src: test/java/org/teiid/common/buffer/impl and 1 other directory.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-07 12:52:06 -0400 (Fri, 07 Oct 2011)
New Revision: 3544
Modified:
trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
trunk/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java
trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java
Log:
TEIID-1750 optimizing sizeutility and correcting the reference initialization logic
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-10-07 15:25:46 UTC (rev 3543)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-10-07 16:52:06 UTC (rev 3544)
@@ -34,6 +34,7 @@
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
+import java.util.ListIterator;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
@@ -87,10 +88,13 @@
String[] types;
private LobManager lobManager;
- private BatchManagerImpl(Long newID, String[] types) {
+ private BatchManagerImpl(Long newID, Class<?>[] types) {
this.id = newID;
this.sizeUtility = new SizeUtility(types);
- this.types = types;
+ this.types = new String[types.length];
+ for (int i = 0; i < types.length; i++) {
+ this.types[i] = DataTypeManager.getDataTypeName(types[i]);
+ }
cache.createCacheGroup(newID);
}
@@ -381,7 +385,7 @@
TupleSourceType tupleSourceType) {
final Long newID = this.tsId.getAndIncrement();
int[] lobIndexes = LobManager.getLobIndexes(elements);
- String[] types = TupleBuffer.getTypeNames(elements);
+ Class<?>[] types = getTypeClasses(elements);
BatchManagerImpl batchManager = createBatchManager(newID, types);
LobManager lobManager = null;
FileStore lobStore = null;
@@ -404,7 +408,7 @@
public STree createSTree(final List elements, String groupName, int keyLength) {
Long newID = this.tsId.getAndIncrement();
int[] lobIndexes = LobManager.getLobIndexes(elements);
- String[] types = TupleBuffer.getTypeNames(elements);
+ Class<?>[] types = getTypeClasses(elements);
BatchManagerImpl bm = createBatchManager(newID, types);
LobManager lobManager = null;
if (lobIndexes != null) {
@@ -422,7 +426,16 @@
return new STree(keyManager, bm, new ListNestedSortComparator(compareIndexes), getProcessorBatchSize(elements), getProcessorBatchSize(elements.subList(0, keyLength)), keyLength, lobManager);
}
- private BatchManagerImpl createBatchManager(final Long newID, String[] types) {
+ private static Class<?>[] getTypeClasses(final List elements) {
+ Class<?>[] types = new Class[elements.size()];
+ for (ListIterator<? extends Expression> i = elements.listIterator(); i.hasNext();) {
+ Expression expr = i.next();
+ types[i.previousIndex()] = expr.getType();
+ }
+ return types;
+ }
+
+ private BatchManagerImpl createBatchManager(final Long newID, Class<?>[] types) {
BatchManagerImpl bm = new BatchManagerImpl(newID, types);
final AtomicBoolean prefersMemory = bm.prefersMemory;
AutoCleanupUtil.setCleanupReference(bm, new Removable() {
@@ -480,12 +493,14 @@
if (this.maxProcessingKBOrig < 0) {
this.maxProcessingKB = Math.max(Math.min(8 * processorBatchSize, Integer.MAX_VALUE), (int)(.1 * maxMemory)/maxActivePlans);
}
- int memoryBatches = (this.maxProcessingKB * maxActivePlans + this.getMaxReserveKB()) / (processorBatchSize * targetBytesPerRow / 1024);
- int logSize = 39 - Integer.numberOfLeadingZeros(memoryBatches);
+ //make a guess at the max number of batches
+ int memoryBatches = maxMemory / (processorBatchSize * targetBytesPerRow / 1024);
+ //memoryBatches represents a full batch, so assume that most will be smaller
+ int logSize = 35 - Integer.numberOfLeadingZeros(memoryBatches);
if (useWeakReferences) {
- weakReferenceCache = new WeakReferenceHashedValueCache<CacheEntry>(Math.min(20, logSize));
+ weakReferenceCache = new WeakReferenceHashedValueCache<CacheEntry>(Math.min(30, logSize));
}
- this.maxSoftReferences = 1 << Math.max(28, logSize+2);
+ this.maxSoftReferences = 1 << Math.min(30, logSize);
}
@Override
@@ -538,8 +553,9 @@
}
void persistBatchReferences() {
- if (activeBatchKB.get() == 0 || activeBatchKB.get() <= reserveBatchKB) {
- int memoryCount = activeBatchKB.get() + getMaxReserveKB() - reserveBatchKB;
+ int activeBatch = activeBatchKB.get();
+ if (activeBatch <= reserveBatchKB) {
+ int memoryCount = activeBatch + getMaxReserveKB() - reserveBatchKB;
if (DataTypeManager.isValueCacheEnabled()) {
if (memoryCount < getMaxReserveKB() / 8) {
DataTypeManager.setValueCacheEnabled(false);
@@ -551,11 +567,11 @@
}
int maxToFree = Math.max(maxProcessingKB>>1, reserveBatchKB>>3);
int freed = 0;
- while (true) {
- if (freed > maxToFree || activeBatchKB.get() == 0 || activeBatchKB.get() < reserveBatchKB * .8) {
+ while (freed <= maxToFree && activeBatchKB.get() > reserveBatchKB * .8) {
+ CacheEntry ce = memoryEntries.evict();
+ if (ce == null) {
break;
}
- CacheEntry ce = memoryEntries.evict();
freed += ce.getSizeEstimate();
activeBatchKB.addAndGet(-ce.getSizeEstimate());
synchronized (ce) {
Modified: trunk/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java
===================================================================
--- trunk/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java 2011-10-07 15:25:46 UTC (rev 3543)
+++ trunk/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java 2011-10-07 16:52:06 UTC (rev 3544)
@@ -26,8 +26,10 @@
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import org.teiid.core.types.DataTypeManager;
@@ -43,7 +45,7 @@
public static final int REFERENCE_SIZE = 8;
private static Map<Class<?>, int[]> SIZE_ESTIMATES = new HashMap<Class<?>, int[]>(128);
-
+ private static Set<Class<?>> VARIABLE_SIZE_TYPES = new HashSet<Class<?>>();
static {
SIZE_ESTIMATES.put(DataTypeManager.DefaultDataClasses.STRING, new int[] {100, 256});
SIZE_ESTIMATES.put(DataTypeManager.DefaultDataClasses.DATE, new int[] {20, 28});
@@ -61,13 +63,17 @@
SIZE_ESTIMATES.put(DataTypeManager.DefaultDataClasses.BOOLEAN, new int[] {1, 1});
SIZE_ESTIMATES.put(DataTypeManager.DefaultDataClasses.BIG_INTEGER, new int[] {75, 100});
SIZE_ESTIMATES.put(DataTypeManager.DefaultDataClasses.BIG_DECIMAL, new int[] {150, 200});
+ VARIABLE_SIZE_TYPES.add(DataTypeManager.DefaultDataClasses.STRING);
+ VARIABLE_SIZE_TYPES.add(DataTypeManager.DefaultDataClasses.OBJECT);
+ VARIABLE_SIZE_TYPES.add(DataTypeManager.DefaultDataClasses.BIG_INTEGER);
+ VARIABLE_SIZE_TYPES.add(DataTypeManager.DefaultDataClasses.BIG_DECIMAL);
}
private long bigIntegerEstimate;
private long bigDecimalEstimate;
- private String[] types;
+ private Class<?>[] types;
- public SizeUtility(String[] types) {
+ public SizeUtility(Class<?>[] types) {
boolean isValueCacheEnabled = DataTypeManager.isValueCacheEnabled();
bigIntegerEstimate = getSize(isValueCacheEnabled, DataTypeManager.DefaultDataClasses.BIG_INTEGER);
bigDecimalEstimate = getSize(isValueCacheEnabled, DataTypeManager.DefaultDataClasses.BIG_DECIMAL);
@@ -83,12 +89,8 @@
// array overhead for all the columns ( 8 object overhead + 4 ref + 4 int)
size += (rowLength * (48 + alignMemory(colLength * REFERENCE_SIZE)));
for (int col = 0; col < colLength; col++) {
- Class<?> type = DataTypeManager.getDataTypeClass(types[col]);
-
- if (type == DataTypeManager.DefaultDataClasses.STRING
- || type == DataTypeManager.DefaultDataClasses.OBJECT
- || type == DataTypeManager.DefaultDataClasses.BIG_INTEGER
- || type == DataTypeManager.DefaultDataClasses.BIG_DECIMAL) {
+ Class<?> type = types[col];
+ if (VARIABLE_SIZE_TYPES.contains(type)) {
int estRow = 0;
for (int row = 0; row < rowLength; row++) {
boolean updateEst = row == estRow;
Modified: trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java
===================================================================
--- trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java 2011-10-07 15:25:46 UTC (rev 3543)
+++ trunk/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java 2011-10-07 16:52:06 UTC (rev 3544)
@@ -34,6 +34,7 @@
import java.util.List;
import org.junit.Test;
+import org.teiid.core.types.DataTypeManager;
public class TestSizeUtility {
@@ -161,7 +162,12 @@
Arrays.asList(new Object[] { "c", new Integer(1), Boolean.FALSE, new Double(0.0), "c", new Integer(1) }) //$NON-NLS-1$ //$NON-NLS-2$
};
- String[] types = {"string", "integer", "boolean", "double", "string", "integer"}; //$NON-NLS-1$//$NON-NLS-2$//$NON-NLS-3$//$NON-NLS-4$ //$NON-NLS-5$//$NON-NLS-6$
+ Class<?>[] types = {DataTypeManager.DefaultDataClasses.STRING,
+ DataTypeManager.DefaultDataClasses.INTEGER,
+ DataTypeManager.DefaultDataClasses.BOOLEAN,
+ DataTypeManager.DefaultDataClasses.DOUBLE,
+ DataTypeManager.DefaultDataClasses.STRING,
+ DataTypeManager.DefaultDataClasses.INTEGER};
long actualSize = new SizeUtility(types).getBatchSize(false, Arrays.asList(expected));
assertEquals("Got unexpected size: ", 2667, actualSize); //$NON-NLS-1$
13 years, 2 months
teiid SVN: r3543 - in branches/7.4.x: documentation/admin-guide/src/main/docbook/en-US/content and 2 other directories.
by teiid-commits@lists.jboss.org
Author: rareddy
Date: 2011-10-07 11:25:46 -0400 (Fri, 07 Oct 2011)
New Revision: 3543
Modified:
branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml
branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/security.xml
branches/7.4.x/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java
branches/7.4.x/runtime/src/test/java/org/teiid/transport/TestCommSockets.java
Log:
SOA-3448: allowing the ability to configure cipher suites for selecting encryption strength. Also has documentation for the LDAP configuration.
Modified: branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml
===================================================================
--- branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml 2011-10-07 14:49:25 UTC (rev 3542)
+++ branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml 2011-10-07 15:25:46 UTC (rev 3543)
@@ -221,6 +221,9 @@
<property name="truststorePassword">passwd</property>
<!-- 1-way, 2-way, anonymous -->
<property name="authenticationMode">anonymous</property>
+ <!-- uncomment for enforcing the minimum 128 bit encryption, edit or supply only supported cipher suites from JVM
+ <property name="enabledCipherSuites">SSL_RSA_WITH_RC4_128_MD5,SSL_RSA_WITH_RC4_128_SHA,SSL_RSA_WITH_3DES_EDE_CBC_SHA,SSL_DHE_DSS_WITH_3DES_EDE_CBC_SHA,SSL_DHE_RSA_WITH_3DES_EDE_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_DSS_WITH_AES_128_CBC_SHA,TLS_KRB5_WITH_RC4_128_MD5,TLS_KRB5_WITH_RC4_128_SHA,TLS_RSA_WITH_AES_128_CBC_SHA,TLS_KRB5_WITH_3DES_EDE_CBC_MD5,TLS_KRB5_WITH_3DES_EDE_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_DSS_WITH_AES_256_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA</property>
+ -->
</bean>
<!-- JDBC Socket connection properties (SSL see below) -->
Modified: branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/security.xml
===================================================================
--- branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/security.xml 2011-10-07 14:49:25 UTC (rev 3542)
+++ branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/security.xml 2011-10-07 15:25:46 UTC (rev 3543)
@@ -84,23 +84,106 @@
The default name of JDBC connection's security-domain is "teiid-security". The default name for Admin connection
is "jmx-console". For the Admin connection's security domain, the user is allowed
to change which LoginModule that "jmx-console" pointing to, however should not change the name of the domain, as this name is
- shared between the "admin-console" application.</para>
+ shared between the "admin-console" application. In existing installations an appropriate security
+ domain may already be configured for use by administrative clients (typically "jmx-console").
+ In this case it may be perfectly valid to reuse this existing security domain instead of
+ creating a new teiid-security security domain.</para>
</note>
<section>
<title>Built-in LoginModules</title>
- <para>JBossAS provides several LoginModules for common authentication needs, such as authenticating from text files or LDAP.</para>
- <para>The UsersRolesLoginModule, which utilizes simple text files
- to authenticate users and to define
- their groups. The teiid-jboss-beans.xml configuration file contains an example of how to use UsersRolesLoginModule.
- Note that this is typically not for production use and is strongly recommended that you replace this login module. Please
- also note that, you can install multiple login modules as part of single security domain configuration and configure them
- to part of login process. For example, for "teiid-security" domain, you can configure a file based and also LDAP based login modules,
- and have your user authenticated with either both or single login module.
+ <para>JBossAS provides several LoginModules for common authentication needs, such as authenticating from a
+ <xref linkend="text-login"/> or a <xref linkend="ldap-login"/>.</para>
+ <para>You can install multiple login modules as part of single security domain configuration and configure them
+ to part of login process. For example, for "teiid-security" domain, you can configure a file based
+ and also LDAP based login modules,
+ and have your user authenticated with either or both login modules. If you want to write your own
+ custom login module, check out the Developer's Guide for instructions.
</para>
- <para>See <ulink url="http://community.jboss.org/docs/DOC-11253">LDAP LoginModule configuration</ulink> for utilizing LDAP based authentication.
- If you want write your own Custom Login module, check out the Developer's Guide for instructions.
- </para>
+
+ <section id="text-login">
+ <title>Text Based LoginModule</title>
+ <para>
+ The UsersRolesLoginModule utilizes simple text files to authenticate users and to define their groups.
+ The teiid-jboss-beans.xml configuration file contains an example of how to use UsersRolesLoginModule.
+ <note>
+ <para>The UsersRolesLoginModule is not recommended for production use and is strongly recommended that you replace this login module.</para>
+ </note>
+ </para>
+ <para>User
+ names and passwords are stored in the <profile>conf/props/teiid-security-users.properties file.
+ <example>
+ <title>Example user.properties file</title>
+ <programlisting><![CDATA[# A users.properties file for use with the UsersRolesLoginModule
+ # username=password
+
+ fred=password
+ george=password
+ ...]]></programlisting>
+ </example>
+
+ JAAS role assignments are stored in the <profile>conf/props/teiid-security-roles.properties file.
+ <example>
+ <title>Example user.properties file</title>
+ <programlisting><![CDATA[# A roles.properties file for use with the UsersRolesLoginModule
+ # username=role1,role2,...
+
+ data_role_1=fred,sally
+ data_role_2=george
+ ]]></programlisting>
+ </example>
+
+ User and role names are entirely up to the needs of the given deployment. For example each application team can set their own security constraints for their VDBs, by mapping their VDB data roles to application specific JAAS roles, e.g. app_role_1=user1,user2,user3.
+ <note>
+ <para>Teiid data roles names are independent of JAAS roles. VDB creators can choose whatever name they want for their data roles, which are then mapped at deployment time to JAAS roles.</para>
+ </note>
+ </para>
+ </section>
+ <section id="ldap-login">
+ <title>LDAP Based LoginModule</title>
+ <para>
+ See
+ <ulink url="http://community.jboss.org/docs/DOC-11253">LDAP LoginModule configuration</ulink>
+ for the AS community guide. The following are streamlined installation instruction.
+ <orderedlist>
+ <listitem>
+ <para>If using SSL to the LDAP server, ensure that the Corporate CA Certificate is added to the JRE trust store.</para>
+ </listitem>
+
+ <listitem>
+ <para>Include LDAP LoginModule in the JAAS Configuration</para>
+ <para>Configure
+ LDAP authentication by editing <profile>conf/login-config.xml. If you wish to configure specifically for teiid, then the security domain teiid-security will need to be created/altered.
+ In new installations the more likely option is that you want to configure LDAP based authentication for the AS itself by modifying the "jmx-console" security domain.
+ You could do one of the following for Teiid:
+ <itemizedlist>
+ <listitem>
+ <para>Reuse the jmx-console (or whatever name you choose) security domain for Teiid by changing the teiid configuration in <profile>/deploy/teiid/teiid-jboss-beans.xml
+ to point to jmx-console, rather than teiid-security.
+ </para>
+ </listitem>
+ <listitem>
+ <para>Follow the same steps to configure an LDAP security domain named teiid-security.
+ </para>
+ </listitem>
+ <listitem>
+ <para>Leave Teiid to use the default file based LoginModule secuirty domain or create an entirely custom security domain configuration.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </para>
+ </listitem>
+ <listitem>
+ <para>Obscure the LDAP Password</para>
+ <para>
+ Finally, protect the password following
+ <ulink url="http://docs.redhat.com/docs/en-US/JBoss_Enterprise_Application_Platform/5...">these instructions.</ulink>
+ Note that the salt must be 8 chars andd see also http://community.jboss.org/message/137756#137756 for more on securing passwords.
+ </para>
+ </listitem>
+ </orderedlist>
+ </para>
+ </section>
</section>
<section>
@@ -250,6 +333,8 @@
<property name="truststorePassword">passwd</property>
<!-- 1-way, 2-way, anonymous -->
<property name="authenticationMode">1-way</property>
+ <!-- an optional property to constrain the cipher suites to be negotiated between server and client -->
+ <property name="enabledCipherSuites">SSL_RSA_WITH_RC4_128_MD5,SSL_RSA_WITH_RC4_128_SHA</property>
</bean>]]></programlisting>
</example>
<itemizedlist>
@@ -270,6 +355,7 @@
public key for the client. Depending upon how you created the keystore and truststores,
this may be same file as defined under "keystoreFilename" property.</para></listitem>
<listitem><para>truststorePassword - password for the truststore. </para></listitem>
+ <listitem><para>enabledCipherSuites - A comma separated list of cipher suites allowed for encryption between server and client. The values must be valid supported cipher suites otherwise SSL connections will fail.</para></listitem>
</itemizedlist>
<section id="ssl_auth">
<title>SSL Authentication Modes</title>
@@ -291,8 +377,9 @@
</section>
<section id="encryption_strength">
<title>Encryption Strength</title>
- <para>Both anonymous SSL and login only encryption are configured to use 128 bit AES encryption.
- 1-way and 2-way SSL allow for cipher suite negotiation based upon the default cipher suites supported by the respective Java platforms of the client and server.
+ <para>Both anonymous SSL and login only encryption are configured to use 128 bit AES encryption by default. By default,
+ 1-way and 2-way SSL allow for cipher suite negotiation based upon the default cipher suites supported by the respective Java platforms of the client and server.
+ User can restrict the cipher suites used for encryption by specifying the <emphasis>enabledCipherSuites</emphasis> property above in ssl configuration.
</para>
</section>
</section>
Modified: branches/7.4.x/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java
===================================================================
--- branches/7.4.x/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java 2011-10-07 14:49:25 UTC (rev 3542)
+++ branches/7.4.x/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java 2011-10-07 15:25:46 UTC (rev 3543)
@@ -59,8 +59,10 @@
private String trustStoreFileName;
private String trustStorePassword = ""; //$NON-NLS-1$
private String authenticationMode = ONEWAY;
+ private String[] enabledCipherSuites;
- public SSLEngine getServerSSLEngine() throws IOException, GeneralSecurityException {
+
+ public SSLEngine getServerSSLEngine() throws IOException, GeneralSecurityException {
if (!isSslEnabled()) {
return null;
}
@@ -86,10 +88,13 @@
if (!(Arrays.asList(result.getSupportedCipherSuites()).contains(SocketUtil.ANON_CIPHER_SUITE))) {
throw new GeneralSecurityException(RuntimePlugin.Util.getString("SSLConfiguration.no_anonymous")); //$NON-NLS-1$
}
- result.setEnabledCipherSuites(new String[] {
- SocketUtil.ANON_CIPHER_SUITE
- });
- }
+ result.setEnabledCipherSuites(new String[] {SocketUtil.ANON_CIPHER_SUITE});
+ } else {
+ if (this.enabledCipherSuites != null) {
+ result.setEnabledCipherSuites(this.enabledCipherSuites);
+ }
+ }
+
result.setNeedClientAuth(TWOWAY.equals(authenticationMode));
return result;
}
@@ -142,4 +147,11 @@
this.authenticationMode = value;
}
+ public void setEnabledCipherSuites(String enabledCipherSuites) {
+ this.enabledCipherSuites = enabledCipherSuites.split(","); //$NON-NLS-1$
+ }
+
+ public String[] getEnabledCipherSuites() {
+ return enabledCipherSuites;
+ }
}
Modified: branches/7.4.x/runtime/src/test/java/org/teiid/transport/TestCommSockets.java
===================================================================
--- branches/7.4.x/runtime/src/test/java/org/teiid/transport/TestCommSockets.java 2011-10-07 14:49:25 UTC (rev 3542)
+++ branches/7.4.x/runtime/src/test/java/org/teiid/transport/TestCommSockets.java 2011-10-07 15:25:46 UTC (rev 3543)
@@ -217,6 +217,7 @@
@Test public void testAnonSSLConnect() throws Exception {
SSLConfiguration config = new SSLConfiguration();
config.setMode(SSLConfiguration.ENABLED);
+ config.setEnabledCipherSuites("x"); //ensure that this cipher suite is not used
config.setAuthenticationMode(SSLConfiguration.ANONYMOUS);
Properties p = new Properties();
p.setProperty("org.teiid.sockets.soTimeout", "100");
@@ -255,4 +256,10 @@
conn.close();
}
+ @Test public void testEnableCipherSuites() throws Exception {
+ SSLConfiguration config = new SSLConfiguration();
+ config.setEnabledCipherSuites("x,y,z");
+ assertArrayEquals(new String[] {"x","y","z"}, config.getEnabledCipherSuites());
+ }
+
}
13 years, 2 months
teiid SVN: r3542 - in branches/7.4.x/engine/src: test/java/org/teiid/query/processor and 1 other directory.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-07 10:49:25 -0400 (Fri, 07 Oct 2011)
New Revision: 3542
Modified:
branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/AccessNode.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java
Log:
TEIID-1778 fix for returning incorrect results
Modified: branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/AccessNode.java
===================================================================
--- branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/AccessNode.java 2011-10-07 12:25:57 UTC (rev 3541)
+++ branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/AccessNode.java 2011-10-07 14:49:25 UTC (rev 3542)
@@ -328,6 +328,15 @@
terminateBatches();
return pullBatch();
}
+
+ @Override
+ protected void addBatchRow(List row) {
+ if (this.getOutputElements().isEmpty()) {
+ //a dummy column was added to the query, just remove it now
+ row = Collections.emptyList();
+ }
+ super.addBatchRow(row);
+ }
private void registerNext() throws TeiidComponentException,
TeiidProcessingException {
Modified: branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java
===================================================================
--- branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java 2011-10-07 12:25:57 UTC (rev 3541)
+++ branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java 2011-10-07 14:49:25 UTC (rev 3542)
@@ -7573,5 +7573,17 @@
Arrays.asList(0, "a")});
}
+ @Test public void testCrossJoinReduction() throws Exception {
+ String sql = "select x.e2 from pm1.g2 y, pm1.g1 x where x.e1 = y.e1 and x.e1 = 'a'"; //$NON-NLS-1$
+
+ HardcodedDataManager dataManager = new HardcodedDataManager();
+ dataManager.addData("SELECT g_0.e2 FROM pm1.g1 AS g_0 WHERE g_0.e1 = 'a'", new List[] {Arrays.asList(3)});
+ dataManager.addData("SELECT 1 FROM pm1.g2 AS g_0 WHERE g_0.e1 = 'a'", new List[] {Arrays.asList(1)});
+ ProcessorPlan plan = helpGetPlan(sql, RealMetadataFactory.example1Cached(), TestOptimizer.getGenericFinder(false));
+
+ helpProcess(plan, dataManager, new List[] {
+ Arrays.asList(3)});
+ }
+
private static final boolean DEBUG = false;
}
13 years, 2 months
teiid SVN: r3541 - in trunk/connectors/translator-jdbc/src: test/java/org/teiid/translator/jdbc/ingres and 1 other directory.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-07 08:25:57 -0400 (Fri, 07 Oct 2011)
New Revision: 3541
Modified:
trunk/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java
trunk/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java
Log:
TEIID-1776 fix for typing issues
Modified: trunk/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java
===================================================================
--- trunk/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java 2011-10-07 01:33:47 UTC (rev 3540)
+++ trunk/connectors/translator-jdbc/src/main/java/org/teiid/translator/jdbc/ingres/IngresExecutionFactory.java 2011-10-07 12:25:57 UTC (rev 3541)
@@ -57,13 +57,14 @@
convert.addTypeMapping("decimal(38,19)", FunctionModifier.BIGDECIMAL); //$NON-NLS-1$
convert.addTypeMapping("decimal(15,0)", FunctionModifier.BIGINTEGER); //$NON-NLS-1$
convert.addTypeMapping("date", FunctionModifier.DATE); //$NON-NLS-1$
- convert.addTypeMapping("time with time zone", FunctionModifier.TIME); //$NON-NLS-1$
- convert.addTypeMapping("timestamp with time zone", FunctionModifier.TIMESTAMP); //$NON-NLS-1$
+ convert.addTypeMapping("time", FunctionModifier.TIME); //$NON-NLS-1$
+ convert.addTypeMapping("timestamp", FunctionModifier.TIMESTAMP); //$NON-NLS-1$
convert.addTypeMapping("char(1)", FunctionModifier.CHAR); //$NON-NLS-1$
convert.addTypeMapping("varchar(4000)", FunctionModifier.STRING); //$NON-NLS-1$
convert.addTypeMapping("blob", FunctionModifier.BLOB); //$NON-NLS-1$
convert.addTypeMapping("clob", FunctionModifier.CLOB); //$NON-NLS-1$
convert.addNumericBooleanConversions();
+ convert.setWideningNumericImplicit(true);
registerFunctionModifier(SourceSystemFunctions.CONVERT, convert);
registerFunctionModifier(SourceSystemFunctions.BITAND, new AliasModifier("bit_and")); //$NON-NLS-1$
Modified: trunk/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java
===================================================================
--- trunk/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java 2011-10-07 01:33:47 UTC (rev 3540)
+++ trunk/connectors/translator-jdbc/src/test/java/org/teiid/translator/jdbc/ingres/TestIngresConvertModifier.java 2011-10-07 12:25:57 UTC (rev 3541)
@@ -68,7 +68,7 @@
}
@Test public void testBooleanToBigDecimal() throws Exception {
- helpTest(LANG_FACTORY.createLiteral(Boolean.TRUE, Boolean.class), "bigdecimal", "cast(1 AS decimal(38,19))"); //$NON-NLS-1$ //$NON-NLS-2$
+ helpTest(LANG_FACTORY.createLiteral(Boolean.TRUE, Boolean.class), "bigdecimal", "1"); //$NON-NLS-1$ //$NON-NLS-2$
}
// Source = BYTE
@@ -82,7 +82,7 @@
}
@Test public void testBigIntegerToDouble() throws Exception {
- helpTest(LANG_FACTORY.createLiteral(new BigInteger("1"), BigInteger.class), "double", "cast(1 AS float)"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
+ helpTest(LANG_FACTORY.createLiteral(new BigInteger("1"), BigInteger.class), "double", "1"); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
}
// Source = FLOAT
@@ -106,7 +106,7 @@
// Source = DATE
@Test public void testDateToTimestamp() throws Exception {
- helpTest(LANG_FACTORY.createLiteral(TimestampUtil.createDate(103, 10, 1), java.sql.Date.class), "timestamp", "cast(DATE '2003-11-01' AS timestamp with time zone)"); //$NON-NLS-1$ //$NON-NLS-2$
+ helpTest(LANG_FACTORY.createLiteral(TimestampUtil.createDate(103, 10, 1), java.sql.Date.class), "timestamp", "cast(DATE '2003-11-01' AS timestamp)"); //$NON-NLS-1$ //$NON-NLS-2$
}
// Source = TIME
13 years, 2 months
teiid SVN: r3540 - in trunk: runtime/src/main/java/org/teiid/transport and 1 other directories.
by teiid-commits@lists.jboss.org
Author: shawkins
Date: 2011-10-06 21:33:47 -0400 (Thu, 06 Oct 2011)
New Revision: 3540
Modified:
trunk/documentation/admin-guide/src/main/docbook/en-US/content/security.xml
trunk/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java
trunk/runtime/src/test/java/org/teiid/transport/TestCommSockets.java
Log:
TEIID-1772 refining cipher suite logic TEIID-1749 expanding the admin guide on login modules
Modified: trunk/documentation/admin-guide/src/main/docbook/en-US/content/security.xml
===================================================================
--- trunk/documentation/admin-guide/src/main/docbook/en-US/content/security.xml 2011-10-06 21:06:06 UTC (rev 3539)
+++ trunk/documentation/admin-guide/src/main/docbook/en-US/content/security.xml 2011-10-07 01:33:47 UTC (rev 3540)
@@ -87,24 +87,80 @@
The default name of JDBC connection's security-domain is "teiid-security". The default name for Admin connection
is "jmx-console". For the Admin connection's security domain, the user is allowed
to change which LoginModule that "jmx-console" pointing to, however should not change the name of the domain, as this name is
- shared between the "admin-console" application.</para>
+ shared between the "admin-console" application. In existing installations an appropriate security domain may already be configured for use by administrative clients (typically "jmx-console").
+ In this case it may be perfectly valid to reuse this existing security domain instead of creating a new teiid-security security domain.</para>
</note>
<section>
<title>Built-in LoginModules</title>
- <para>JBossAS provides several LoginModules for common authentication needs, such as authenticating from text files or LDAP.</para>
- <para>The UsersRolesLoginModule, which utilizes simple text files
- to authenticate users and to define
- their groups. The teiid-jboss-beans.xml configuration file contains an example of how to use UsersRolesLoginModule.
- Note that this is typically not for production use and is strongly recommended that you replace this login module. Please
- also note that, you can install multiple login modules as part of single security domain configuration and configure them
+ <para>JBossAS provides several LoginModules for common authentication needs, such as authenticating from a <xref linkend="text-login"/> or a <xref linkend="ldap-login"/>.</para>
+ <para>You can install multiple login modules as part of single security domain configuration and configure them
to part of login process. For example, for "teiid-security" domain, you can configure a file based and also LDAP based login modules,
- and have your user authenticated with either both or single login module.
+ and have your user authenticated with either or both login modules. If you want to write your own custom login module, check out the Developer's Guide for instructions.
</para>
- <para>See <ulink url="http://community.jboss.org/docs/DOC-11253">LDAP LoginModule configuration</ulink> for utilizing LDAP based authentication.
- If you want write your own Custom Login module, check out the Developer's Guide for instructions.
- </para>
+
+ <section id="text-login">
+ <title>Text Based LoginModule</title>
+ <para>The UsersRolesLoginModule utilizes simple text files to authenticate users and to define their groups.
+The teiid-jboss-beans.xml configuration file contains an example of how to use UsersRolesLoginModule.
+<note><para>The UsersRolesLoginModule is not recommended for production use and is strongly recommended that you replace this login module.</para></note>
+ </para>
+ <para>User names and passwords are stored in the <profile>conf/props/teiid-security-users.properties file.
+<example><title>Example user.properties file</title>
+<programlisting><![CDATA[# A users.properties file for use with the UsersRolesLoginModule
+# username=password
+
+fred=password
+george=password
+...]]></programlisting></example>
+
+JAAS role assignments are stored in the <profile>conf/props/teiid-security-roles.properties file.
+<example><title>Example user.properties file</title>
+<programlisting><![CDATA[# A roles.properties file for use with the UsersRolesLoginModule
+# username=role1,role2,...
+
+data_role_1=fred,sally
+data_role_2=george
+]]></programlisting></example>
+
+User and role names are entirely up to the needs of the given deployment. For example each application team can set their own security constraints for their VDBs, by mapping their VDB data roles to application specific JAAS roles, e.g. app_role_1=user1,user2,user3.
+<note><para>Teiid data roles names are independent of JAAS roles. VDB creators can choose whatever name they want for their data roles, which are then mapped at deployment time to JAAS roles.</para></note>
+</para>
</section>
+
+ <section id="ldap-login">
+ <title>LDAP Based LoginModule</title>
+ <para>See <ulink url="http://community.jboss.org/docs/DOC-11253">LDAP LoginModule configuration</ulink> for the AS community guide. The following are streamlined installation instruction.
+ <orderedlist>
+ <listitem><para>If using SSL to the LDAP server, ensure that the Corporate CA Certificate is added to the JRE trust store.</para>
+ </listitem>
+
+ <listitem><para>Include LDAP LoginModule in the JAAS Configuration</para>
+ <para>Configure LDAP authentication by editing <profile>conf/login-config.xml. If you wish to configure specifically for teiid, then the security domain teiid-security will need to be created/altered.
+ In new installations the more likely option is that you want to configure LDAP based authentication for the AS itself by modifying the "jmx-console" security domain.
+ You could do one of the following for Teiid:
+ <itemizedlist>
+ <listitem>
+ <para>Reuse the jmx-console (or whatever name you choose) security domain for Teiid by changing the teiid configuration &jboss-beans; to point to jmx-console, rather than teiid-security.
+ </para>
+ </listitem>
+ <listitem>
+ <para>Follow the same steps to configure an LDAP security domain named teiid-security.
+ </para>
+ </listitem>
+ <listitem>
+ <para>Leave Teiid to use the default file based LoginModule secuirty domain or create an entirely custom security domain configuration.
+ </para>
+ </listitem>
+ </itemizedlist>
+ </para></listitem>
+ <listitem><para>Obscure the LDAP Password</para><para>Finally, protect the password following <ulink url="http://docs.redhat.com/docs/en-US/JBoss_Enterprise_Application_Platform/5...">these instructions.</ulink>
+ Note that the salt must be 8 chars andd see also http://community.jboss.org/message/137756#137756 for more on securing passwords.
+ </para></listitem>
+ </orderedlist>
+ </para>
+ </section>
+ </section>
<section>
<title>Kerberos support through GSSAPI</title>
@@ -389,7 +445,7 @@
public key for the client. Depending upon how you created the keystore and truststores,
this may be same file as defined under "keystoreFilename" property.</para></listitem>
<listitem><para>truststorePassword - password for the truststore. </para></listitem>
- <listitem><para>enabledCipherSuites - A comma separated list of cipher suites allowed for encryption between server and client</para></listitem>
+ <listitem><para>enabledCipherSuites - A comma separated list of cipher suites allowed for encryption between server and client. The values must be valid supported cipher suites otherwise SSL connections will fail.</para></listitem>
</itemizedlist>
<section id="ssl_auth">
<title>SSL Authentication Modes</title>
Modified: trunk/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java
===================================================================
--- trunk/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java 2011-10-06 21:06:06 UTC (rev 3539)
+++ trunk/runtime/src/main/java/org/teiid/transport/SSLConfiguration.java 2011-10-07 01:33:47 UTC (rev 3540)
@@ -24,9 +24,7 @@
import java.io.IOException;
import java.security.GeneralSecurityException;
-import java.util.ArrayList;
import java.util.Arrays;
-import java.util.StringTokenizer;
import javax.net.ssl.KeyManagerFactory;
import javax.net.ssl.SSLContext;
@@ -90,7 +88,7 @@
if (!(Arrays.asList(result.getSupportedCipherSuites()).contains(SocketUtil.ANON_CIPHER_SUITE))) {
throw new GeneralSecurityException(RuntimePlugin.Util.getString("SSLConfiguration.no_anonymous")); //$NON-NLS-1$
}
- result.setEnabledCipherSuites(this.enabledCipherSuites == null?new String[] {SocketUtil.ANON_CIPHER_SUITE}:this.enabledCipherSuites);
+ result.setEnabledCipherSuites(new String[] {SocketUtil.ANON_CIPHER_SUITE});
} else {
if (this.enabledCipherSuites != null) {
result.setEnabledCipherSuites(this.enabledCipherSuites);
@@ -150,14 +148,10 @@
}
public void setEnabledCipherSuites(String enabledCipherSuites) {
- ArrayList<String> ciphers = new ArrayList<String>();
- StringTokenizer st = new StringTokenizer(enabledCipherSuites);
- while(st.hasMoreTokens()) {
- ciphers.add(st.nextToken().trim());
- }
-
- if (!ciphers.isEmpty()) {
- this.enabledCipherSuites = ciphers.toArray(new String[ciphers.size()]);
- }
+ this.enabledCipherSuites = enabledCipherSuites.split(","); //$NON-NLS-1$
}
+
+ public String[] getEnabledCipherSuites() {
+ return enabledCipherSuites;
+ }
}
Modified: trunk/runtime/src/test/java/org/teiid/transport/TestCommSockets.java
===================================================================
--- trunk/runtime/src/test/java/org/teiid/transport/TestCommSockets.java 2011-10-06 21:06:06 UTC (rev 3539)
+++ trunk/runtime/src/test/java/org/teiid/transport/TestCommSockets.java 2011-10-07 01:33:47 UTC (rev 3540)
@@ -217,6 +217,7 @@
@Test public void testAnonSSLConnect() throws Exception {
SSLConfiguration config = new SSLConfiguration();
config.setMode(SSLConfiguration.ENABLED);
+ config.setEnabledCipherSuites("x"); //ensure that this cipher suite is not used
config.setAuthenticationMode(SSLConfiguration.ANONYMOUS);
Properties p = new Properties();
p.setProperty("org.teiid.sockets.soTimeout", "100");
@@ -255,4 +256,10 @@
conn.close();
}
+ @Test public void testEnableCipherSuites() throws Exception {
+ SSLConfiguration config = new SSLConfiguration();
+ config.setEnabledCipherSuites("x,y,z");
+ assertArrayEquals(new String[] {"x","y","z"}, config.getEnabledCipherSuites());
+ }
+
}
13 years, 2 months